From e19f10e5371fa492082e3393d1240ff735342c42 Mon Sep 17 00:00:00 2001 From: comphead Date: Thu, 6 Nov 2025 17:00:38 -0800 Subject: [PATCH 1/6] chore: Fallback to Spark for Windows --- .../scala/org/apache/comet/CometConf.scala | 2 +- docs/source/user-guide/latest/configs.md | 34 +++---- .../apache/comet/CometExpressionSuite.scala | 24 +---- .../comet/exec/CometAggregateSuite.scala | 53 ----------- .../apache/comet/exec/CometExecSuite.scala | 4 +- .../comet/exec/CometWindowExecSuite.scala | 91 ++++++++++++++++--- 6 files changed, 100 insertions(+), 108 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index d48d149728..d44a19a072 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -251,7 +251,7 @@ object CometConf extends ShimCometConf { val COMET_EXEC_EXPAND_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("expand", defaultValue = true) val COMET_EXEC_WINDOW_ENABLED: ConfigEntry[Boolean] = - createExecEnabledConfig("window", defaultValue = true) + createExecEnabledConfig("window", defaultValue = false) val COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("takeOrderedAndProject", defaultValue = true) diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 6caaa53b1b..4a70234f45 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -139,23 +139,23 @@ These settings can be used to determine which parts of the plan are accelerated | Config | Description | Default Value | |--------|-------------|---------------| -| `spark.comet.exec.aggregate.enabled` | Whether to enable aggregate by default. | true | -| `spark.comet.exec.broadcastExchange.enabled` | Whether to enable broadcastExchange by default. | true | -| `spark.comet.exec.broadcastHashJoin.enabled` | Whether to enable broadcastHashJoin by default. | true | -| `spark.comet.exec.coalesce.enabled` | Whether to enable coalesce by default. | true | -| `spark.comet.exec.collectLimit.enabled` | Whether to enable collectLimit by default. | true | -| `spark.comet.exec.expand.enabled` | Whether to enable expand by default. | true | -| `spark.comet.exec.filter.enabled` | Whether to enable filter by default. | true | -| `spark.comet.exec.globalLimit.enabled` | Whether to enable globalLimit by default. | true | -| `spark.comet.exec.hashJoin.enabled` | Whether to enable hashJoin by default. | true | -| `spark.comet.exec.localLimit.enabled` | Whether to enable localLimit by default. | true | -| `spark.comet.exec.project.enabled` | Whether to enable project by default. | true | -| `spark.comet.exec.sort.enabled` | Whether to enable sort by default. | true | -| `spark.comet.exec.sortMergeJoin.enabled` | Whether to enable sortMergeJoin by default. | true | -| `spark.comet.exec.sortMergeJoinWithJoinFilter.enabled` | Experimental support for Sort Merge Join with filter | false | -| `spark.comet.exec.takeOrderedAndProject.enabled` | Whether to enable takeOrderedAndProject by default. | true | -| `spark.comet.exec.union.enabled` | Whether to enable union by default. | true | -| `spark.comet.exec.window.enabled` | Whether to enable window by default. | true | +| `spark.comet.exec.aggregate.enabled` | Whether to enable aggregate by default. | true | +| `spark.comet.exec.broadcastExchange.enabled` | Whether to enable broadcastExchange by default. | true | +| `spark.comet.exec.broadcastHashJoin.enabled` | Whether to enable broadcastHashJoin by default. | true | +| `spark.comet.exec.coalesce.enabled` | Whether to enable coalesce by default. | true | +| `spark.comet.exec.collectLimit.enabled` | Whether to enable collectLimit by default. | true | +| `spark.comet.exec.expand.enabled` | Whether to enable expand by default. | true | +| `spark.comet.exec.filter.enabled` | Whether to enable filter by default. | true | +| `spark.comet.exec.globalLimit.enabled` | Whether to enable globalLimit by default. | true | +| `spark.comet.exec.hashJoin.enabled` | Whether to enable hashJoin by default. | true | +| `spark.comet.exec.localLimit.enabled` | Whether to enable localLimit by default. | true | +| `spark.comet.exec.project.enabled` | Whether to enable project by default. | true | +| `spark.comet.exec.sort.enabled` | Whether to enable sort by default. | true | +| `spark.comet.exec.sortMergeJoin.enabled` | Whether to enable sortMergeJoin by default. | true | +| `spark.comet.exec.sortMergeJoinWithJoinFilter.enabled` | Experimental support for Sort Merge Join with filter | false | +| `spark.comet.exec.takeOrderedAndProject.enabled` | Whether to enable takeOrderedAndProject by default. | true | +| `spark.comet.exec.union.enabled` | Whether to enable union by default. | true | +| `spark.comet.exec.window.enabled` | Whether to enable window by default. | false | ## Enabling or Disabling Individual Scalar Expressions diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index e5d2b395a0..e4fb08101d 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -30,10 +30,9 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Literal, TruncDate, TruncTimestamp} import org.apache.spark.sql.catalyst.optimizer.SimplifyExtractValueOps -import org.apache.spark.sql.comet.{CometColumnarToRowExec, CometProjectExec, CometWindowExec} +import org.apache.spark.sql.comet.{CometColumnarToRowExec, CometProjectExec} import org.apache.spark.sql.execution.{InputAdapter, ProjectExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper -import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.SESSION_LOCAL_TIMEZONE @@ -3097,27 +3096,6 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - test("window query with rangeBetween") { - - // values are int - val df = Seq(1, 2, 4, 3, 2, 1).toDF("value") - val window = Window.orderBy($"value".desc) - - // ranges are long - val df2 = df.select( - $"value", - sum($"value").over(window.rangeBetween(Window.unboundedPreceding, 1L)), - sum($"value").over(window.rangeBetween(1L, Window.unboundedFollowing))) - - // Comet does not support RANGE BETWEEN - // https://github.com/apache/datafusion-comet/issues/1246 - val (_, cometPlan) = checkSparkAnswer(df2) - val cometWindowExecs = collect(cometPlan) { case w: CometWindowExec => - w - } - assert(cometWindowExecs.isEmpty) - } - test("vectorized reader: missing all struct fields") { Seq(true, false).foreach { offheapEnabled => withSQLConf( diff --git a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala index f38dfb8ea5..5fdd24c46d 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometAggregateSuite.scala @@ -25,7 +25,6 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, DataFrame, Row} import org.apache.spark.sql.catalyst.optimizer.EliminateSorts import org.apache.spark.sql.comet.CometHashAggregateExec -import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper import org.apache.spark.sql.functions.{avg, count_distinct, sum} import org.apache.spark.sql.internal.SQLConf @@ -94,58 +93,6 @@ class CometAggregateSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } - test("lead/lag should return the default value if the offset row does not exist") { - withSQLConf( - CometConf.COMET_ENABLED.key -> "true", - CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", - CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { - checkSparkAnswer(sql(""" - |SELECT - | lag(123, 100, 321) OVER (ORDER BY id) as lag, - | lead(123, 100, 321) OVER (ORDER BY id) as lead - |FROM (SELECT 1 as id) tmp - """.stripMargin)) - - checkSparkAnswer(sql(""" - |SELECT - | lag(123, 100, a) OVER (ORDER BY id) as lag, - | lead(123, 100, a) OVER (ORDER BY id) as lead - |FROM (SELECT 1 as id, 2 as a) tmp - """.stripMargin)) - } - } - - // based on Spark's SQLWindowFunctionSuite test of the same name - test("window function: partition and order expressions") { - for (shuffleMode <- Seq("auto", "native", "jvm")) { - withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> shuffleMode) { - val df = - Seq((1, "a", 5), (2, "a", 6), (3, "b", 7), (4, "b", 8), (5, "c", 9), (6, "c", 10)).toDF( - "month", - "area", - "product") - df.createOrReplaceTempView("windowData") - val df2 = sql(""" - |select month, area, product, sum(product + 1) over (partition by 1 order by 2) - |from windowData - """.stripMargin) - checkSparkAnswer(df2) - val cometShuffles = collect(df2.queryExecution.executedPlan) { - case _: CometShuffleExchangeExec => true - } - if (shuffleMode == "jvm" || shuffleMode == "auto") { - assert(cometShuffles.length == 1) - } else { - // we fall back to Spark for shuffle because we do not support - // native shuffle with a LocalTableScan input, and we do not fall - // back to Comet columnar shuffle due to - // https://github.com/apache/datafusion-comet/issues/1248 - assert(cometShuffles.isEmpty) - } - } - } - } - test("multiple column distinct count") { withSQLConf( CometConf.COMET_ENABLED.key -> "true", diff --git a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala index cba0329df8..73bcf94558 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometExecSuite.scala @@ -1697,7 +1697,9 @@ class CometExecSuite extends CometTestBase { test("TakeOrderedAndProjectExec") { Seq("true", "false").foreach(aqeEnabled => - withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled) { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> aqeEnabled, + CometConf.COMET_EXEC_WINDOW_ENABLED.key -> "true") { withTable("t1") { val numRows = 10 spark diff --git a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala index 488829f996..79e3497fda 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala @@ -24,8 +24,10 @@ import org.scalatest.Tag import org.apache.hadoop.fs.Path import org.apache.spark.sql.{CometTestBase, Row} +import org.apache.spark.sql.comet.CometWindowExec +import org.apache.spark.sql.comet.execution.shuffle.CometShuffleExchangeExec import org.apache.spark.sql.expressions.Window -import org.apache.spark.sql.functions.{count, lead} +import org.apache.spark.sql.functions.{count, lead, sum} import org.apache.spark.sql.internal.SQLConf import org.apache.comet.CometConf @@ -39,12 +41,86 @@ class CometWindowExecSuite extends CometTestBase { super.test(testName, testTags: _*) { withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_EXEC_WINDOW_ENABLED.key -> "true", CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_AUTO) { testFun } } } + test("lead/lag should return the default value if the offset row does not exist") { + withSQLConf( + CometConf.COMET_ENABLED.key -> "true", + CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", + CometConf.COMET_SHUFFLE_MODE.key -> "jvm") { + checkSparkAnswer(sql(""" + |SELECT + | lag(123, 100, 321) OVER (ORDER BY id) as lag, + | lead(123, 100, 321) OVER (ORDER BY id) as lead + |FROM (SELECT 1 as id) tmp + """.stripMargin)) + + checkSparkAnswer(sql(""" + |SELECT + | lag(123, 100, a) OVER (ORDER BY id) as lag, + | lead(123, 100, a) OVER (ORDER BY id) as lead + |FROM (SELECT 1 as id, 2 as a) tmp + """.stripMargin)) + } + } + + test("window query with rangeBetween") { + + // values are int + val df = Seq(1, 2, 4, 3, 2, 1).toDF("value") + val window = Window.orderBy($"value".desc) + + // ranges are long + val df2 = df.select( + $"value", + sum($"value").over(window.rangeBetween(Window.unboundedPreceding, 1L)), + sum($"value").over(window.rangeBetween(1L, Window.unboundedFollowing))) + + // Comet does not support RANGE BETWEEN + // https://github.com/apache/datafusion-comet/issues/1246 + val (_, cometPlan) = checkSparkAnswer(df2) + val cometWindowExecs = collect(cometPlan) { case w: CometWindowExec => + w + } + assert(cometWindowExecs.isEmpty) + } + + // based on Spark's SQLWindowFunctionSuite test of the same name + test("window function: partition and order expressions") { + for (shuffleMode <- Seq("auto", "native", "jvm")) { + withSQLConf(CometConf.COMET_SHUFFLE_MODE.key -> shuffleMode) { + val df = + Seq((1, "a", 5), (2, "a", 6), (3, "b", 7), (4, "b", 8), (5, "c", 9), (6, "c", 10)).toDF( + "month", + "area", + "product") + df.createOrReplaceTempView("windowData") + val df2 = sql(""" + |select month, area, product, sum(product + 1) over (partition by 1 order by 2) + |from windowData + """.stripMargin) + checkSparkAnswer(df2) + val cometShuffles = collect(df2.queryExecution.executedPlan) { + case _: CometShuffleExchangeExec => true + } + if (shuffleMode == "jvm" || shuffleMode == "auto") { + assert(cometShuffles.length == 1) + } else { + // we fall back to Spark for shuffle because we do not support + // native shuffle with a LocalTableScan input, and we do not fall + // back to Comet columnar shuffle due to + // https://github.com/apache/datafusion-comet/issues/1248 + assert(cometShuffles.isEmpty) + } + } + } + } + test( "fall back to Spark when the partition spec and order spec are not the same for window function") { withTempView("test") { @@ -289,7 +365,7 @@ class CometWindowExecSuite extends CometTestBase { // TODO: COUNT with ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW produces incorrect results // Returns wrong cnt values - ordering issue causes swapped values for rows with same partition - ignore("window: COUNT with ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW") { + test("window: COUNT with ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) @@ -310,7 +386,6 @@ class CometWindowExecSuite extends CometTestBase { } // TODO: SUM with ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING produces incorrect results - // Returns wrong sum_c values - ordering issue causes swapped values for rows with same partition ignore("window: SUM with ROWS BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING") { withTempDir { dir => (0 until 30) @@ -354,7 +429,6 @@ class CometWindowExecSuite extends CometTestBase { } // TODO: SUM with ROWS BETWEEN produces incorrect results - // Returns wrong sum_c values for some rows ignore("window: SUM with ROWS BETWEEN 2 PRECEDING AND CURRENT ROW") { withTempDir { dir => (0 until 30) @@ -530,7 +604,6 @@ class CometWindowExecSuite extends CometTestBase { } // TODO: LAG produces incorrect results - // Returns wrong lag_c values - ordering issue in results ignore("window: LAG with default offset") { withTempDir { dir => (0 until 30) @@ -552,7 +625,6 @@ class CometWindowExecSuite extends CometTestBase { } // TODO: LAG with offset 2 produces incorrect results - // Returns wrong lag_c_2 values - ordering issue in results ignore("window: LAG with offset 2 and default value") { withTempDir { dir => (0 until 30) @@ -574,7 +646,6 @@ class CometWindowExecSuite extends CometTestBase { } // TODO: LEAD produces incorrect results - // Returns wrong lead_c values - ordering issue in results ignore("window: LEAD with default offset") { withTempDir { dir => (0 until 30) @@ -596,7 +667,6 @@ class CometWindowExecSuite extends CometTestBase { } // TODO: LEAD with offset 2 produces incorrect results - // Returns wrong lead_c_2 values - ordering issue in results ignore("window: LEAD with offset 2 and default value") { withTempDir { dir => (0 until 30) @@ -662,7 +732,6 @@ class CometWindowExecSuite extends CometTestBase { } // TODO: NTH_VALUE returns incorrect results - produces 0 instead of null for first row, - // and incorrect values for subsequent rows in partition ignore("window: NTH_VALUE with position 2") { withTempDir { dir => (0 until 30) @@ -706,7 +775,6 @@ class CometWindowExecSuite extends CometTestBase { } // TODO: Multiple window functions with mixed frame types (RowFrame and RangeFrame) - // produces incorrect row_num values - ordering issue in results ignore("window: multiple window functions in single query") { withTempDir { dir => (0 until 30) @@ -933,7 +1001,6 @@ class CometWindowExecSuite extends CometTestBase { } // TODO: ROWS BETWEEN with negative offset produces incorrect results - // Returns wrong values for avg_c calculation ignore("window: ROWS BETWEEN with negative offset") { withTempDir { dir => (0 until 30) @@ -955,7 +1022,6 @@ class CometWindowExecSuite extends CometTestBase { } // TODO: All ranking functions together produce incorrect row_num values - // Ordering issue causes row numbers to be swapped for rows with same partition/order values ignore("window: all ranking functions together") { withTempDir { dir => (0 until 30) @@ -980,5 +1046,4 @@ class CometWindowExecSuite extends CometTestBase { checkSparkAnswerAndOperator(df) } } - } From 481b68871d32be0042e6d7da8a173711bfb83b03 Mon Sep 17 00:00:00 2001 From: comphead Date: Thu, 6 Nov 2025 17:07:00 -0800 Subject: [PATCH 2/6] chore: Fallback to Spark for Windows --- .../test/scala/org/apache/comet/exec/CometWindowExecSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala index 79e3497fda..a0d3d733ff 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala @@ -365,7 +365,7 @@ class CometWindowExecSuite extends CometTestBase { // TODO: COUNT with ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW produces incorrect results // Returns wrong cnt values - ordering issue causes swapped values for rows with same partition - test("window: COUNT with ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW") { + ignore("window: COUNT with ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW") { withTempDir { dir => (0 until 30) .map(i => (i % 3, i % 5, i)) From 7dccd13631171a1775bcb28d7264b2fd086bbad4 Mon Sep 17 00:00:00 2001 From: comphead Date: Fri, 7 Nov 2025 13:50:13 -0800 Subject: [PATCH 3/6] chore: Fallback to Spark for windows --- .../q12.native_iceberg_compat/explain.txt | 26 +- .../q12.native_iceberg_compat/extended.txt | 10 +- .../q12.native_iceberg_compat/simplified.txt | 74 +-- .../q12/explain.txt | 26 +- .../q12/extended.txt | 10 +- .../q12/simplified.txt | 74 +-- .../q20.native_iceberg_compat/explain.txt | 26 +- .../q20.native_iceberg_compat/extended.txt | 10 +- .../q20.native_iceberg_compat/simplified.txt | 74 +-- .../q20/explain.txt | 26 +- .../q20/extended.txt | 10 +- .../q20/simplified.txt | 74 +-- .../q36.native_iceberg_compat/explain.txt | 26 +- .../q36.native_iceberg_compat/extended.txt | 10 +- .../q36.native_iceberg_compat/simplified.txt | 88 ++- .../q36/explain.txt | 26 +- .../q36/extended.txt | 10 +- .../q36/simplified.txt | 88 ++- .../q47.native_iceberg_compat/explain.txt | 214 +++--- .../q47.native_iceberg_compat/extended.txt | 154 +++-- .../q47.native_iceberg_compat/simplified.txt | 130 ++-- .../q47/explain.txt | 214 +++--- .../q47/extended.txt | 154 +++-- .../q47/simplified.txt | 130 ++-- .../q49.native_iceberg_compat/explain.txt | 327 +++++----- .../q49.native_iceberg_compat/extended.txt | 177 +++-- .../q49.native_iceberg_compat/simplified.txt | 189 +++--- .../q49/explain.txt | 327 +++++----- .../q49/extended.txt | 177 +++-- .../q49/simplified.txt | 189 +++--- .../q51.native_iceberg_compat/explain.txt | 186 +++--- .../q51.native_iceberg_compat/extended.txt | 94 ++- .../q51.native_iceberg_compat/simplified.txt | 124 ++-- .../q51/explain.txt | 186 +++--- .../q51/extended.txt | 94 ++- .../q51/simplified.txt | 124 ++-- .../q53.native_iceberg_compat/explain.txt | 30 +- .../q53.native_iceberg_compat/extended.txt | 12 +- .../q53.native_iceberg_compat/simplified.txt | 86 ++- .../q53/explain.txt | 30 +- .../q53/extended.txt | 12 +- .../q53/simplified.txt | 86 ++- .../q57.native_iceberg_compat/explain.txt | 214 +++--- .../q57.native_iceberg_compat/extended.txt | 154 +++-- .../q57.native_iceberg_compat/simplified.txt | 130 ++-- .../q57/explain.txt | 214 +++--- .../q57/extended.txt | 154 +++-- .../q57/simplified.txt | 130 ++-- .../q63.native_iceberg_compat/explain.txt | 30 +- .../q63.native_iceberg_compat/extended.txt | 12 +- .../q63.native_iceberg_compat/simplified.txt | 86 ++- .../q63/explain.txt | 30 +- .../q63/extended.txt | 12 +- .../q63/simplified.txt | 86 ++- .../q70.native_iceberg_compat/explain.txt | 26 +- .../q70.native_iceberg_compat/extended.txt | 10 +- .../q70.native_iceberg_compat/simplified.txt | 146 ++--- .../q70/explain.txt | 26 +- .../q70/extended.txt | 10 +- .../q70/simplified.txt | 146 ++--- .../q86.native_iceberg_compat/explain.txt | 26 +- .../q86.native_iceberg_compat/extended.txt | 10 +- .../q86.native_iceberg_compat/simplified.txt | 76 ++- .../q86/explain.txt | 26 +- .../q86/extended.txt | 10 +- .../q86/simplified.txt | 76 ++- .../q89.native_iceberg_compat/explain.txt | 30 +- .../q89.native_iceberg_compat/extended.txt | 12 +- .../q89.native_iceberg_compat/simplified.txt | 86 ++- .../q89/explain.txt | 30 +- .../q89/extended.txt | 12 +- .../q89/simplified.txt | 86 ++- .../q98.native_iceberg_compat/explain.txt | 94 ++- .../q98.native_iceberg_compat/extended.txt | 61 +- .../q98.native_iceberg_compat/simplified.txt | 73 +-- .../q98/explain.txt | 94 ++- .../q98/extended.txt | 61 +- .../q98/simplified.txt | 73 +-- .../q12.native_iceberg_compat/explain.txt | 26 +- .../q12.native_iceberg_compat/extended.txt | 10 +- .../q12.native_iceberg_compat/simplified.txt | 68 +- .../approved-plans-v1_4/q12/explain.txt | 26 +- .../approved-plans-v1_4/q12/extended.txt | 10 +- .../approved-plans-v1_4/q12/simplified.txt | 68 +- .../q20.native_iceberg_compat/explain.txt | 26 +- .../q20.native_iceberg_compat/extended.txt | 10 +- .../q20.native_iceberg_compat/simplified.txt | 68 +- .../approved-plans-v1_4/q20/explain.txt | 26 +- .../approved-plans-v1_4/q20/extended.txt | 10 +- .../approved-plans-v1_4/q20/simplified.txt | 68 +- .../q36.native_iceberg_compat/explain.txt | 26 +- .../q36.native_iceberg_compat/extended.txt | 10 +- .../q36.native_iceberg_compat/simplified.txt | 82 ++- .../approved-plans-v1_4/q36/explain.txt | 26 +- .../approved-plans-v1_4/q36/extended.txt | 10 +- .../approved-plans-v1_4/q36/simplified.txt | 82 ++- .../q44.native_iceberg_compat/explain.txt | 195 +++--- .../q44.native_iceberg_compat/extended.txt | 115 ++-- .../q44.native_iceberg_compat/simplified.txt | 103 ++- .../approved-plans-v1_4/q44/explain.txt | 195 +++--- .../approved-plans-v1_4/q44/extended.txt | 115 ++-- .../approved-plans-v1_4/q44/simplified.txt | 103 ++- .../q47.native_iceberg_compat/explain.txt | 208 +++--- .../q47.native_iceberg_compat/extended.txt | 150 +++-- .../q47.native_iceberg_compat/simplified.txt | 122 ++-- .../approved-plans-v1_4/q47/explain.txt | 208 +++--- .../approved-plans-v1_4/q47/extended.txt | 150 +++-- .../approved-plans-v1_4/q47/simplified.txt | 122 ++-- .../q49.native_iceberg_compat/explain.txt | 327 +++++----- .../q49.native_iceberg_compat/extended.txt | 177 +++-- .../q49.native_iceberg_compat/simplified.txt | 189 +++--- .../approved-plans-v1_4/q49/explain.txt | 327 +++++----- .../approved-plans-v1_4/q49/extended.txt | 177 +++-- .../approved-plans-v1_4/q49/simplified.txt | 189 +++--- .../q51.native_iceberg_compat/explain.txt | 180 +++-- .../q51.native_iceberg_compat/extended.txt | 90 ++- .../q51.native_iceberg_compat/simplified.txt | 112 ++-- .../approved-plans-v1_4/q51/explain.txt | 180 +++-- .../approved-plans-v1_4/q51/extended.txt | 90 ++- .../approved-plans-v1_4/q51/simplified.txt | 112 ++-- .../q53.native_iceberg_compat/explain.txt | 30 +- .../q53.native_iceberg_compat/extended.txt | 12 +- .../q53.native_iceberg_compat/simplified.txt | 80 ++- .../approved-plans-v1_4/q53/explain.txt | 30 +- .../approved-plans-v1_4/q53/extended.txt | 12 +- .../approved-plans-v1_4/q53/simplified.txt | 80 ++- .../q57.native_iceberg_compat/explain.txt | 208 +++--- .../q57.native_iceberg_compat/extended.txt | 150 +++-- .../q57.native_iceberg_compat/simplified.txt | 122 ++-- .../approved-plans-v1_4/q57/explain.txt | 208 +++--- .../approved-plans-v1_4/q57/extended.txt | 150 +++-- .../approved-plans-v1_4/q57/simplified.txt | 122 ++-- .../q63.native_iceberg_compat/explain.txt | 30 +- .../q63.native_iceberg_compat/extended.txt | 12 +- .../q63.native_iceberg_compat/simplified.txt | 80 ++- .../approved-plans-v1_4/q63/explain.txt | 30 +- .../approved-plans-v1_4/q63/extended.txt | 12 +- .../approved-plans-v1_4/q63/simplified.txt | 80 ++- .../q67.native_iceberg_compat/explain.txt | 24 +- .../q67.native_iceberg_compat/extended.txt | 10 +- .../q67.native_iceberg_compat/simplified.txt | 82 ++- .../approved-plans-v1_4/q67/explain.txt | 24 +- .../approved-plans-v1_4/q67/extended.txt | 10 +- .../approved-plans-v1_4/q67/simplified.txt | 82 ++- .../q70.native_iceberg_compat/explain.txt | 258 ++++---- .../q70.native_iceberg_compat/extended.txt | 110 ++-- .../q70.native_iceberg_compat/simplified.txt | 128 ++-- .../approved-plans-v1_4/q70/explain.txt | 258 ++++---- .../approved-plans-v1_4/q70/extended.txt | 110 ++-- .../approved-plans-v1_4/q70/simplified.txt | 128 ++-- .../q86.native_iceberg_compat/explain.txt | 26 +- .../q86.native_iceberg_compat/extended.txt | 10 +- .../q86.native_iceberg_compat/simplified.txt | 70 +- .../approved-plans-v1_4/q86/explain.txt | 26 +- .../approved-plans-v1_4/q86/extended.txt | 10 +- .../approved-plans-v1_4/q86/simplified.txt | 70 +- .../q89.native_iceberg_compat/explain.txt | 30 +- .../q89.native_iceberg_compat/extended.txt | 12 +- .../q89.native_iceberg_compat/simplified.txt | 80 ++- .../approved-plans-v1_4/q89/explain.txt | 30 +- .../approved-plans-v1_4/q89/extended.txt | 12 +- .../approved-plans-v1_4/q89/simplified.txt | 80 ++- .../q98.native_iceberg_compat/explain.txt | 92 ++- .../q98.native_iceberg_compat/extended.txt | 59 +- .../q98.native_iceberg_compat/simplified.txt | 67 +- .../approved-plans-v1_4/q98/explain.txt | 92 ++- .../approved-plans-v1_4/q98/extended.txt | 59 +- .../approved-plans-v1_4/q98/simplified.txt | 67 +- .../q12.native_iceberg_compat/explain.txt | 26 +- .../q12.native_iceberg_compat/extended.txt | 10 +- .../q12.native_iceberg_compat/simplified.txt | 74 +-- .../q12/explain.txt | 26 +- .../q12/extended.txt | 10 +- .../q12/simplified.txt | 74 +-- .../q20.native_iceberg_compat/explain.txt | 26 +- .../q20.native_iceberg_compat/extended.txt | 10 +- .../q20.native_iceberg_compat/simplified.txt | 74 +-- .../q20/explain.txt | 26 +- .../q20/extended.txt | 10 +- .../q20/simplified.txt | 74 +-- .../q36a.native_iceberg_compat/explain.txt | 26 +- .../q36a.native_iceberg_compat/extended.txt | 10 +- .../q36a.native_iceberg_compat/simplified.txt | 142 ++-- .../q36a/explain.txt | 26 +- .../q36a/extended.txt | 10 +- .../q36a/simplified.txt | 142 ++-- .../q47.native_iceberg_compat/explain.txt | 214 +++--- .../q47.native_iceberg_compat/extended.txt | 154 +++-- .../q47.native_iceberg_compat/simplified.txt | 130 ++-- .../q47/explain.txt | 214 +++--- .../q47/extended.txt | 154 +++-- .../q47/simplified.txt | 130 ++-- .../q49.native_iceberg_compat/explain.txt | 327 +++++----- .../q49.native_iceberg_compat/extended.txt | 177 +++-- .../q49.native_iceberg_compat/simplified.txt | 189 +++--- .../q49/explain.txt | 327 +++++----- .../q49/extended.txt | 177 +++-- .../q49/simplified.txt | 189 +++--- .../q51a.native_iceberg_compat/explain.txt | 613 ++++++++---------- .../q51a.native_iceberg_compat/extended.txt | 419 ++++++------ .../q51a.native_iceberg_compat/simplified.txt | 204 +++--- .../q51a/explain.txt | 613 ++++++++---------- .../q51a/extended.txt | 419 ++++++------ .../q51a/simplified.txt | 204 +++--- .../q57.native_iceberg_compat/explain.txt | 214 +++--- .../q57.native_iceberg_compat/extended.txt | 154 +++-- .../q57.native_iceberg_compat/simplified.txt | 130 ++-- .../q57/explain.txt | 214 +++--- .../q57/extended.txt | 154 +++-- .../q57/simplified.txt | 130 ++-- .../q70a.native_iceberg_compat/explain.txt | 26 +- .../q70a.native_iceberg_compat/extended.txt | 10 +- .../q70a.native_iceberg_compat/simplified.txt | 200 +++--- .../q70a/explain.txt | 26 +- .../q70a/extended.txt | 10 +- .../q70a/simplified.txt | 200 +++--- .../q86a.native_iceberg_compat/explain.txt | 26 +- .../q86a.native_iceberg_compat/extended.txt | 10 +- .../q86a.native_iceberg_compat/simplified.txt | 130 ++-- .../q86a/explain.txt | 26 +- .../q86a/extended.txt | 10 +- .../q86a/simplified.txt | 130 ++-- .../q98.native_iceberg_compat/explain.txt | 90 ++- .../q98.native_iceberg_compat/extended.txt | 61 +- .../q98.native_iceberg_compat/simplified.txt | 73 +-- .../q98/explain.txt | 90 ++- .../q98/extended.txt | 61 +- .../q98/simplified.txt | 73 +-- .../q12.native_iceberg_compat/explain.txt | 26 +- .../q12.native_iceberg_compat/extended.txt | 10 +- .../q12.native_iceberg_compat/simplified.txt | 68 +- .../approved-plans-v2_7/q12/explain.txt | 26 +- .../approved-plans-v2_7/q12/extended.txt | 10 +- .../approved-plans-v2_7/q12/simplified.txt | 68 +- .../q20.native_iceberg_compat/explain.txt | 26 +- .../q20.native_iceberg_compat/extended.txt | 10 +- .../q20.native_iceberg_compat/simplified.txt | 68 +- .../approved-plans-v2_7/q20/explain.txt | 26 +- .../approved-plans-v2_7/q20/extended.txt | 10 +- .../approved-plans-v2_7/q20/simplified.txt | 68 +- .../q36a.native_iceberg_compat/explain.txt | 26 +- .../q36a.native_iceberg_compat/extended.txt | 10 +- .../q36a.native_iceberg_compat/simplified.txt | 108 ++- .../approved-plans-v2_7/q36a/explain.txt | 26 +- .../approved-plans-v2_7/q36a/extended.txt | 10 +- .../approved-plans-v2_7/q36a/simplified.txt | 108 ++- .../q47.native_iceberg_compat/explain.txt | 208 +++--- .../q47.native_iceberg_compat/extended.txt | 150 +++-- .../q47.native_iceberg_compat/simplified.txt | 122 ++-- .../approved-plans-v2_7/q47/explain.txt | 208 +++--- .../approved-plans-v2_7/q47/extended.txt | 150 +++-- .../approved-plans-v2_7/q47/simplified.txt | 122 ++-- .../q49.native_iceberg_compat/explain.txt | 327 +++++----- .../q49.native_iceberg_compat/extended.txt | 177 +++-- .../q49.native_iceberg_compat/simplified.txt | 189 +++--- .../approved-plans-v2_7/q49/explain.txt | 327 +++++----- .../approved-plans-v2_7/q49/extended.txt | 177 +++-- .../approved-plans-v2_7/q49/simplified.txt | 189 +++--- .../q51a.native_iceberg_compat/explain.txt | 595 ++++++++--------- .../q51a.native_iceberg_compat/extended.txt | 395 ++++++----- .../q51a.native_iceberg_compat/simplified.txt | 194 +++--- .../approved-plans-v2_7/q51a/explain.txt | 595 ++++++++--------- .../approved-plans-v2_7/q51a/extended.txt | 395 ++++++----- .../approved-plans-v2_7/q51a/simplified.txt | 194 +++--- .../q57.native_iceberg_compat/explain.txt | 208 +++--- .../q57.native_iceberg_compat/extended.txt | 150 +++-- .../q57.native_iceberg_compat/simplified.txt | 122 ++-- .../approved-plans-v2_7/q57/explain.txt | 208 +++--- .../approved-plans-v2_7/q57/extended.txt | 150 +++-- .../approved-plans-v2_7/q57/simplified.txt | 122 ++-- .../q67a.native_iceberg_compat/explain.txt | 24 +- .../q67a.native_iceberg_compat/extended.txt | 10 +- .../q67a.native_iceberg_compat/simplified.txt | 162 +++-- .../approved-plans-v2_7/q67a/explain.txt | 24 +- .../approved-plans-v2_7/q67a/extended.txt | 10 +- .../approved-plans-v2_7/q67a/simplified.txt | 162 +++-- .../q70a.native_iceberg_compat/explain.txt | 395 +++++------ .../q70a.native_iceberg_compat/extended.txt | 318 +++++---- .../q70a.native_iceberg_compat/simplified.txt | 164 ++--- .../approved-plans-v2_7/q70a/explain.txt | 395 +++++------ .../approved-plans-v2_7/q70a/extended.txt | 318 +++++---- .../approved-plans-v2_7/q70a/simplified.txt | 164 ++--- .../q86a.native_iceberg_compat/explain.txt | 26 +- .../q86a.native_iceberg_compat/extended.txt | 10 +- .../q86a.native_iceberg_compat/simplified.txt | 96 ++- .../approved-plans-v2_7/q86a/explain.txt | 26 +- .../approved-plans-v2_7/q86a/extended.txt | 10 +- .../approved-plans-v2_7/q86a/simplified.txt | 96 ++- .../q98.native_iceberg_compat/explain.txt | 88 ++- .../q98.native_iceberg_compat/extended.txt | 59 +- .../q98.native_iceberg_compat/simplified.txt | 67 +- .../approved-plans-v2_7/q98/explain.txt | 88 ++- .../approved-plans-v2_7/q98/extended.txt | 59 +- .../approved-plans-v2_7/q98/simplified.txt | 67 +- 294 files changed, 14924 insertions(+), 17028 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt index 866cc628a9..f4001679f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometProject (22) + +- CometWindowExec (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] +(22) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(24) TakeOrderedAndProject +(23) CometTakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] + +(24) CometColumnarToRow [codegen id : 2] +Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt index 0ab9094c86..6cb7b610ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] @@ -29,4 +29,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt index 1fdffc3b73..24bb6d630c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt index 866cc628a9..f4001679f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometProject (22) + +- CometWindowExec (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] +(22) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(24) TakeOrderedAndProject +(23) CometTakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] + +(24) CometColumnarToRow [codegen id : 2] +Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt index 0ab9094c86..6cb7b610ee 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] @@ -29,4 +29,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt index 1fdffc3b73..24bb6d630c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt index 88752d6e12..38b20b7f0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometProject (22) + +- CometWindowExec (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] +(22) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(24) TakeOrderedAndProject +(23) CometTakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] + +(24) CometColumnarToRow [codegen id : 2] +Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt index 5944fee1b0..f856bbd3b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] @@ -29,4 +29,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt index db00a01c2d..2cb011147e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt index 88752d6e12..38b20b7f0c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometProject (22) + +- CometWindowExec (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] +(22) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(24) TakeOrderedAndProject +(23) CometTakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] + +(24) CometColumnarToRow [codegen id : 2] +Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt index 5944fee1b0..f856bbd3b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] @@ -29,4 +29,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt index db00a01c2d..2cb011147e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt index 3c6285353c..aedafeeaf5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (31) -+- * Project (30) - +- Window (29) - +- * CometColumnarToRow (28) +* CometColumnarToRow (31) ++- CometTakeOrderedAndProject (30) + +- CometProject (29) + +- CometWindowExec (28) +- CometSort (27) +- CometColumnarExchange (26) +- * HashAggregate (25) @@ -159,20 +159,20 @@ Arguments: hashpartitioning(_w1#26, _w2#27, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27], [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 ASC NULLS FIRST] -(28) CometColumnarToRow [codegen id : 2] +(28) CometWindowExec Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28], [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 ASC NULLS FIRST] -(29) Window -Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 ASC NULLS FIRST] - -(30) Project [codegen id : 3] -Output [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +(29) CometProject Input [8]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] +Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] + +(30) CometTakeOrderedAndProject +Input [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#24 DESC NULLS LAST,CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST,rank_within_parent#28 ASC NULLS FIRST], output=[gross_margin#23,i_category#16,i_class#17,lochierarchy#24,rank_within_parent#28]), [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28], 100, 0, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] -(31) TakeOrderedAndProject +(31) CometColumnarToRow [codegen id : 2] Input [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] -Arguments: 100, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt index 83d5fb61ec..63b74d54c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: Unsupported result expressions found in: List((MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS gross_margin#5, i_category#6, i_class#7, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS lochierarchy#9, (MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS _w0#10, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS _w1#11, CASE WHEN (cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint) = 0) THEN i_category#6 END AS _w2#12)] @@ -36,4 +36,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 28 out of 34 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 34 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt index 6e4a6a969b..2b4c1272a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt @@ -1,45 +1,43 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (3) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] - CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt index 3c6285353c..aedafeeaf5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (31) -+- * Project (30) - +- Window (29) - +- * CometColumnarToRow (28) +* CometColumnarToRow (31) ++- CometTakeOrderedAndProject (30) + +- CometProject (29) + +- CometWindowExec (28) +- CometSort (27) +- CometColumnarExchange (26) +- * HashAggregate (25) @@ -159,20 +159,20 @@ Arguments: hashpartitioning(_w1#26, _w2#27, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27], [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 ASC NULLS FIRST] -(28) CometColumnarToRow [codegen id : 2] +(28) CometWindowExec Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28], [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 ASC NULLS FIRST] -(29) Window -Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 ASC NULLS FIRST] - -(30) Project [codegen id : 3] -Output [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +(29) CometProject Input [8]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] +Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] + +(30) CometTakeOrderedAndProject +Input [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#24 DESC NULLS LAST,CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST,rank_within_parent#28 ASC NULLS FIRST], output=[gross_margin#23,i_category#16,i_class#17,lochierarchy#24,rank_within_parent#28]), [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28], 100, 0, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] -(31) TakeOrderedAndProject +(31) CometColumnarToRow [codegen id : 2] Input [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] -Arguments: 100, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt index 83d5fb61ec..63b74d54c8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: Unsupported result expressions found in: List((MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS gross_margin#5, i_category#6, i_class#7, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS lochierarchy#9, (MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS _w0#10, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS _w1#11, CASE WHEN (cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint) = 0) THEN i_category#6 END AS _w2#12)] @@ -36,4 +36,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 28 out of 34 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 34 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt index 6e4a6a969b..2b4c1272a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt @@ -1,45 +1,43 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (3) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] - CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt index 52a3eee555..ab01108c6c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt @@ -1,54 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (29) + : : +- CometFilter (28) + : : +- CometWindowExec (27) + : : +- CometFilter (26) + : : +- CometWindowExec (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- CometBroadcastExchange (37) + : +- CometProject (36) + : +- CometWindowExec (35) + : +- CometSort (34) + : +- CometColumnarExchange (33) + : +- * HashAggregate (32) + : +- * CometColumnarToRow (31) + : +- ReusedExchange (30) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -166,136 +160,112 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 2] +(25) CometWindowExec Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(27) Filter [codegen id : 3] +(26) CometFilter Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(28) Window +(27) CometWindowExec Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22], [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(29) Filter [codegen id : 10] +(28) CometFilter Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(30) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] +(29) CometProject Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -(31) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(32) CometColumnarToRow [codegen id : 4] +(31) CometColumnarToRow [codegen id : 2] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(33) HashAggregate [codegen id : 4] +(32) HashAggregate [codegen id : 2] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] -(34) CometColumnarExchange +(33) CometColumnarExchange Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(35) CometSort +(34) CometSort Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] - -(37) Window +(35) CometWindowExec Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32], [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(36) CometProject Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(39) BroadcastExchange +(37) CometBroadcastExchange Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] +Right output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)], Inner, BuildRight -(41) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] +(39) CometProject Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -(42) ReusedExchange [Reuses operator id: 34] -Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] - -(43) CometSort -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 8] -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] - -(45) Window -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] - -(46) Project [codegen id : 9] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] +(40) ReusedExchange [Reuses operator id: 37] +Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -(47) BroadcastExchange -Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] +Right output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#38 - 1)], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 1)] -Join type: Inner -Join condition: None +(42) CometProject +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#37 AS nsum#40] -(49) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(43) CometTakeOrderedAndProject +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,s_store_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,s_store_name#15,s_company_name#16,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -(50) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +(44) CometColumnarToRow [codegen id : 4] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(53) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) BroadcastExchange +(48) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt index 03f1eab564..eeae7c1e95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt @@ -1,80 +1,78 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] @@ -105,4 +103,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt index 0c8d77b89c..a72267567e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt @@ -1,77 +1,57 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (10) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (3) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) +WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] CometColumnarToRow InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt index 52a3eee555..ab01108c6c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt @@ -1,54 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (29) + : : +- CometFilter (28) + : : +- CometWindowExec (27) + : : +- CometFilter (26) + : : +- CometWindowExec (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- CometBroadcastExchange (37) + : +- CometProject (36) + : +- CometWindowExec (35) + : +- CometSort (34) + : +- CometColumnarExchange (33) + : +- * HashAggregate (32) + : +- * CometColumnarToRow (31) + : +- ReusedExchange (30) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -166,136 +160,112 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 2] +(25) CometWindowExec Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(27) Filter [codegen id : 3] +(26) CometFilter Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(28) Window +(27) CometWindowExec Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22], [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(29) Filter [codegen id : 10] +(28) CometFilter Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(30) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] +(29) CometProject Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -(31) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(32) CometColumnarToRow [codegen id : 4] +(31) CometColumnarToRow [codegen id : 2] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(33) HashAggregate [codegen id : 4] +(32) HashAggregate [codegen id : 2] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] -(34) CometColumnarExchange +(33) CometColumnarExchange Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(35) CometSort +(34) CometSort Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] - -(37) Window +(35) CometWindowExec Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32], [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(36) CometProject Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(39) BroadcastExchange +(37) CometBroadcastExchange Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] +Right output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)], Inner, BuildRight -(41) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] +(39) CometProject Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -(42) ReusedExchange [Reuses operator id: 34] -Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] - -(43) CometSort -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 8] -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] - -(45) Window -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] - -(46) Project [codegen id : 9] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] +(40) ReusedExchange [Reuses operator id: 37] +Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -(47) BroadcastExchange -Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] +Right output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#38 - 1)], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 1)] -Join type: Inner -Join condition: None +(42) CometProject +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#37 AS nsum#40] -(49) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(43) CometTakeOrderedAndProject +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,s_store_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,s_store_name#15,s_company_name#16,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -(50) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +(44) CometColumnarToRow [codegen id : 4] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(53) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) BroadcastExchange +(48) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt index 03f1eab564..eeae7c1e95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt @@ -1,80 +1,78 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] @@ -105,4 +103,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt index 0c8d77b89c..a72267567e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt @@ -1,77 +1,57 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (10) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (3) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) +WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] CometColumnarToRow InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt index 338f5f4201..9c336d7e3c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt @@ -1,82 +1,79 @@ == Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) +* CometColumnarToRow (75) ++- CometTakeOrderedAndProject (74) + +- CometHashAggregate (73) + +- CometExchange (72) + +- CometHashAggregate (71) + +- CometUnion (70) + :- CometProject (25) + : +- CometFilter (24) + : +- CometWindowExec (23) + : +- CometSort (22) + : +- CometWindowExec (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- CometProject (47) + : +- CometFilter (46) + : +- CometWindowExec (45) + : +- CometSort (44) + : +- CometWindowExec (43) + : +- CometSort (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometExchange (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometBroadcastExchange (29) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (35) + +- CometProject (69) + +- CometFilter (68) + +- CometWindowExec (67) + +- CometSort (66) + +- CometWindowExec (65) + +- CometSort (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometProject (56) + : +- CometBroadcastHashJoin (55) + : :- CometBroadcastExchange (51) + : : +- CometProject (50) + : : +- CometFilter (49) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) + +- ReusedExchange (57) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -173,30 +170,27 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 1] +(21) CometWindowExec Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] +(22) CometSort Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] -(24) Window +(23) CometWindowExec Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(25) Filter [codegen id : 3] +(24) CometFilter Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(25) CometProject Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -204,100 +198,97 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(29) CometProject +(28) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometBroadcastExchange +(29) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(33) CometProject +(32) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(34) CometBroadcastHashJoin +(33) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(35) CometProject +(34) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(36) ReusedExchange [Reuses operator id: 13] +(35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(37) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(38) CometProject +(37) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(39) CometHashAggregate +(38) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(40) CometExchange +(39) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(41) CometHashAggregate +(40) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(42) CometExchange +(41) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort +(42) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window +(43) CometWindowExec Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(46) Sort [codegen id : 5] +(44) CometSort Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] -(47) Window +(45) CometWindowExec Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(48) Filter [codegen id : 6] +(46) CometFilter Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +(47) CometProject Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -305,158 +296,156 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter +(49) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(52) CometProject +(50) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(53) CometBroadcastExchange +(51) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(56) CometProject +(54) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(57) CometBroadcastHashJoin +(55) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(58) CometProject +(56) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(59) ReusedExchange [Reuses operator id: 13] +(57) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(60) CometBroadcastHashJoin +(58) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(61) CometProject +(59) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(62) CometHashAggregate +(60) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(61) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(64) CometHashAggregate +(62) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(65) CometExchange +(63) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) CometSort +(64) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) CometColumnarToRow [codegen id : 7] +(65) CometWindowExec Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] +(66) CometSort Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] -(70) Window +(67) CometWindowExec Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(71) Filter [codegen id : 9] +(68) CometFilter Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(69) CometProject Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(73) Union +(70) CometUnion +Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(74) HashAggregate [codegen id : 10] +(71) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarExchange +(72) CometExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(76) CometHashAggregate +(73) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(77) CometTakeOrderedAndProject +(74) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(78) CometColumnarToRow [codegen id : 11] +(75) CometColumnarToRow [codegen id : 1] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter +(77) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(81) CometProject +(78) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(79) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(83) BroadcastExchange +(80) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt index 4988a26564..6a8de7fe28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt @@ -1,94 +1,91 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt index ca80833ee5..4f128e83de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt @@ -1,110 +1,87 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometUnion [channel,item,return_ratio,return_rank,currency_rank] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt index 338f5f4201..9c336d7e3c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt @@ -1,82 +1,79 @@ == Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) +* CometColumnarToRow (75) ++- CometTakeOrderedAndProject (74) + +- CometHashAggregate (73) + +- CometExchange (72) + +- CometHashAggregate (71) + +- CometUnion (70) + :- CometProject (25) + : +- CometFilter (24) + : +- CometWindowExec (23) + : +- CometSort (22) + : +- CometWindowExec (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- CometProject (47) + : +- CometFilter (46) + : +- CometWindowExec (45) + : +- CometSort (44) + : +- CometWindowExec (43) + : +- CometSort (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometExchange (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometBroadcastExchange (29) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (35) + +- CometProject (69) + +- CometFilter (68) + +- CometWindowExec (67) + +- CometSort (66) + +- CometWindowExec (65) + +- CometSort (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometProject (56) + : +- CometBroadcastHashJoin (55) + : :- CometBroadcastExchange (51) + : : +- CometProject (50) + : : +- CometFilter (49) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) + +- ReusedExchange (57) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -173,30 +170,27 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 1] +(21) CometWindowExec Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] +(22) CometSort Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] -(24) Window +(23) CometWindowExec Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(25) Filter [codegen id : 3] +(24) CometFilter Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(25) CometProject Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -204,100 +198,97 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(29) CometProject +(28) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometBroadcastExchange +(29) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(33) CometProject +(32) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(34) CometBroadcastHashJoin +(33) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(35) CometProject +(34) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(36) ReusedExchange [Reuses operator id: 13] +(35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(37) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(38) CometProject +(37) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(39) CometHashAggregate +(38) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(40) CometExchange +(39) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(41) CometHashAggregate +(40) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(42) CometExchange +(41) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort +(42) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window +(43) CometWindowExec Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(46) Sort [codegen id : 5] +(44) CometSort Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] -(47) Window +(45) CometWindowExec Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(48) Filter [codegen id : 6] +(46) CometFilter Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +(47) CometProject Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -305,158 +296,156 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter +(49) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(52) CometProject +(50) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(53) CometBroadcastExchange +(51) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(56) CometProject +(54) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(57) CometBroadcastHashJoin +(55) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(58) CometProject +(56) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(59) ReusedExchange [Reuses operator id: 13] +(57) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(60) CometBroadcastHashJoin +(58) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(61) CometProject +(59) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(62) CometHashAggregate +(60) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(61) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(64) CometHashAggregate +(62) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(65) CometExchange +(63) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) CometSort +(64) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) CometColumnarToRow [codegen id : 7] +(65) CometWindowExec Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] +(66) CometSort Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] -(70) Window +(67) CometWindowExec Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(71) Filter [codegen id : 9] +(68) CometFilter Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(69) CometProject Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(73) Union +(70) CometUnion +Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(74) HashAggregate [codegen id : 10] +(71) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarExchange +(72) CometExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(76) CometHashAggregate +(73) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(77) CometTakeOrderedAndProject +(74) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(78) CometColumnarToRow [codegen id : 11] +(75) CometColumnarToRow [codegen id : 1] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter +(77) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(81) CometProject +(78) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(79) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(83) BroadcastExchange +(80) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt index 4988a26564..6a8de7fe28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt @@ -1,94 +1,91 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt index ca80833ee5..4f128e83de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt @@ -1,110 +1,87 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometUnion [channel,item,return_ratio,return_rank,currency_rank] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt index bb1b8f3167..c6e6ed1565 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt @@ -1,47 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Filter (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometProject (37) - +- CometSortMergeJoin (36) - :- CometSort (19) - : +- CometColumnarExchange (18) - : +- * Project (17) - : +- Window (16) - : +- * CometColumnarToRow (15) - : +- CometSort (14) - : +- CometColumnarExchange (13) - : +- * HashAggregate (12) - : +- * CometColumnarToRow (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (35) - +- CometColumnarExchange (34) - +- * Project (33) - +- Window (32) - +- * CometColumnarToRow (31) - +- CometSort (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * CometColumnarToRow (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (20) - +- ReusedExchange (22) +* CometColumnarToRow (41) ++- CometTakeOrderedAndProject (40) + +- CometFilter (39) + +- CometWindowExec (38) + +- CometSort (37) + +- CometExchange (36) + +- CometProject (35) + +- CometSortMergeJoin (34) + :- CometSort (18) + : +- CometExchange (17) + : +- CometProject (16) + : +- CometWindowExec (15) + : +- CometSort (14) + : +- CometColumnarExchange (13) + : +- * HashAggregate (12) + : +- * CometColumnarToRow (11) + : +- CometExchange (10) + : +- CometHashAggregate (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- CometSort (33) + +- CometExchange (32) + +- CometProject (31) + +- CometWindowExec (30) + +- CometSort (29) + +- CometColumnarExchange (28) + +- * HashAggregate (27) + +- * CometColumnarToRow (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometFilter (20) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) + +- ReusedExchange (21) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -111,26 +109,23 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) CometColumnarToRow [codegen id : 2] +(15) CometWindowExec Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12], [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(16) Window -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(17) Project [codegen id : 3] -Output [3]: [item_sk#10, d_date#6, cume_sales#12] +(16) CometProject Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] +Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10, d_date#6, cume_sales#12] -(18) CometColumnarExchange +(17) CometExchange Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(19) CometSort +(18) CometSort Input [3]: [item_sk#10, d_date#6, cume_sales#12] Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] @@ -138,132 +133,129 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(21) CometFilter +(20) CometFilter Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Condition : isnotnull(ss_item_sk#13) -(22) ReusedExchange [Reuses operator id: 6] +(21) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#17, d_date#18] -(23) CometBroadcastHashJoin +(22) CometBroadcastHashJoin Left output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Right output [2]: [d_date_sk#17, d_date#18] Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight -(24) CometProject +(23) CometProject Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] Arguments: [ss_item_sk#13, ss_sales_price#14, d_date#18], [ss_item_sk#13, ss_sales_price#14, d_date#18] -(25) CometHashAggregate +(24) CometHashAggregate Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -(26) CometExchange +(25) CometExchange Input [3]: [ss_item_sk#13, d_date#18, sum#19] Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(27) CometColumnarToRow [codegen id : 4] +(26) CometColumnarToRow [codegen id : 2] Input [3]: [ss_item_sk#13, d_date#18, sum#19] -(28) HashAggregate [codegen id : 4] +(27) HashAggregate [codegen id : 2] Input [3]: [ss_item_sk#13, d_date#18, sum#19] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] Results [4]: [ss_item_sk#13 AS item_sk#21, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS _w0#22, ss_item_sk#13] -(29) CometColumnarExchange +(28) CometColumnarExchange Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(30) CometSort +(29) CometSort Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] -(31) CometColumnarToRow [codegen id : 5] -Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] - -(32) Window +(30) CometWindowExec Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] -Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] +Arguments: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23], [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] -(33) Project [codegen id : 6] -Output [3]: [item_sk#21, d_date#18, cume_sales#23] +(31) CometProject Input [5]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23] +Arguments: [item_sk#21, d_date#18, cume_sales#23], [item_sk#21, d_date#18, cume_sales#23] -(34) CometColumnarExchange +(32) CometExchange Input [3]: [item_sk#21, d_date#18, cume_sales#23] -Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(35) CometSort +(33) CometSort Input [3]: [item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#21, d_date#18, cume_sales#23], [item_sk#21 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] -(36) CometSortMergeJoin +(34) CometSortMergeJoin Left output [3]: [item_sk#10, d_date#6, cume_sales#12] Right output [3]: [item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#10, d_date#6], [item_sk#21, d_date#18], FullOuter -(37) CometProject +(35) CometProject Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#21 END AS item_sk#24, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#25, cume_sales#12 AS web_sales#26, cume_sales#23 AS store_sales#27] -(38) CometExchange +(36) CometExchange Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(39) CometSort +(37) CometSort Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] -(40) CometColumnarToRow [codegen id : 7] +(38) CometWindowExec Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29], [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] -(41) Window -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] - -(42) Filter [codegen id : 8] +(39) CometFilter Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) -(43) TakeOrderedAndProject +(40) CometTakeOrderedAndProject +Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#24 ASC NULLS FIRST,d_date#25 ASC NULLS FIRST], output=[item_sk#24,d_date#25,web_sales#26,store_sales#27,web_cumulative#28,store_cumulative#29]), [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29], 100, 0, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] + +(41) CometColumnarToRow [codegen id : 3] Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter +(43) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(46) CometProject +(44) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(47) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(48) BroadcastExchange +(46) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt index c74adc3809..ff86e0f44c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt @@ -1,55 +1,53 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS _w0#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS _w0#6, ws_item_sk#1)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS _w0#12, ss_item_sk#7)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS _w0#12, ss_item_sk#7)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 47 eligible operators (74%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 47 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt index dc6a7ea61f..493a61c885 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt @@ -1,68 +1,56 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (8) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (3) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (6) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #8 - WholeStageCodegen (4) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #6 +WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometWindowExec [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #2 + CometProject [item_sk,d_date,cume_sales] + CometWindowExec [item_sk,d_date,_w0,ws_item_sk,cume_sales] + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [ws_item_sk,d_date] #4 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #6 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #7 + CometProject [item_sk,d_date,cume_sales] + CometWindowExec [item_sk,d_date,_w0,ss_item_sk,cume_sales] + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometColumnarExchange [ss_item_sk] #8 + WholeStageCodegen (2) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [ss_item_sk,d_date] #9 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt index bb1b8f3167..c6e6ed1565 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt @@ -1,47 +1,45 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Filter (42) - +- Window (41) - +- * CometColumnarToRow (40) - +- CometSort (39) - +- CometExchange (38) - +- CometProject (37) - +- CometSortMergeJoin (36) - :- CometSort (19) - : +- CometColumnarExchange (18) - : +- * Project (17) - : +- Window (16) - : +- * CometColumnarToRow (15) - : +- CometSort (14) - : +- CometColumnarExchange (13) - : +- * HashAggregate (12) - : +- * CometColumnarToRow (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (35) - +- CometColumnarExchange (34) - +- * Project (33) - +- Window (32) - +- * CometColumnarToRow (31) - +- CometSort (30) - +- CometColumnarExchange (29) - +- * HashAggregate (28) - +- * CometColumnarToRow (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometProject (24) - +- CometBroadcastHashJoin (23) - :- CometFilter (21) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (20) - +- ReusedExchange (22) +* CometColumnarToRow (41) ++- CometTakeOrderedAndProject (40) + +- CometFilter (39) + +- CometWindowExec (38) + +- CometSort (37) + +- CometExchange (36) + +- CometProject (35) + +- CometSortMergeJoin (34) + :- CometSort (18) + : +- CometExchange (17) + : +- CometProject (16) + : +- CometWindowExec (15) + : +- CometSort (14) + : +- CometColumnarExchange (13) + : +- * HashAggregate (12) + : +- * CometColumnarToRow (11) + : +- CometExchange (10) + : +- CometHashAggregate (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- CometSort (33) + +- CometExchange (32) + +- CometProject (31) + +- CometWindowExec (30) + +- CometSort (29) + +- CometColumnarExchange (28) + +- * HashAggregate (27) + +- * CometColumnarToRow (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometFilter (20) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) + +- ReusedExchange (21) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -111,26 +109,23 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) CometColumnarToRow [codegen id : 2] +(15) CometWindowExec Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12], [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(16) Window -Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(17) Project [codegen id : 3] -Output [3]: [item_sk#10, d_date#6, cume_sales#12] +(16) CometProject Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] +Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10, d_date#6, cume_sales#12] -(18) CometColumnarExchange +(17) CometExchange Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(19) CometSort +(18) CometSort Input [3]: [item_sk#10, d_date#6, cume_sales#12] Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] @@ -138,132 +133,129 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(21) CometFilter +(20) CometFilter Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Condition : isnotnull(ss_item_sk#13) -(22) ReusedExchange [Reuses operator id: 6] +(21) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#17, d_date#18] -(23) CometBroadcastHashJoin +(22) CometBroadcastHashJoin Left output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Right output [2]: [d_date_sk#17, d_date#18] Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight -(24) CometProject +(23) CometProject Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] Arguments: [ss_item_sk#13, ss_sales_price#14, d_date#18], [ss_item_sk#13, ss_sales_price#14, d_date#18] -(25) CometHashAggregate +(24) CometHashAggregate Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -(26) CometExchange +(25) CometExchange Input [3]: [ss_item_sk#13, d_date#18, sum#19] Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(27) CometColumnarToRow [codegen id : 4] +(26) CometColumnarToRow [codegen id : 2] Input [3]: [ss_item_sk#13, d_date#18, sum#19] -(28) HashAggregate [codegen id : 4] +(27) HashAggregate [codegen id : 2] Input [3]: [ss_item_sk#13, d_date#18, sum#19] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] Results [4]: [ss_item_sk#13 AS item_sk#21, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS _w0#22, ss_item_sk#13] -(29) CometColumnarExchange +(28) CometColumnarExchange Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(30) CometSort +(29) CometSort Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] -(31) CometColumnarToRow [codegen id : 5] -Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] - -(32) Window +(30) CometWindowExec Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] -Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] +Arguments: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23], [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] -(33) Project [codegen id : 6] -Output [3]: [item_sk#21, d_date#18, cume_sales#23] +(31) CometProject Input [5]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23] +Arguments: [item_sk#21, d_date#18, cume_sales#23], [item_sk#21, d_date#18, cume_sales#23] -(34) CometColumnarExchange +(32) CometExchange Input [3]: [item_sk#21, d_date#18, cume_sales#23] -Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(35) CometSort +(33) CometSort Input [3]: [item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#21, d_date#18, cume_sales#23], [item_sk#21 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] -(36) CometSortMergeJoin +(34) CometSortMergeJoin Left output [3]: [item_sk#10, d_date#6, cume_sales#12] Right output [3]: [item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#10, d_date#6], [item_sk#21, d_date#18], FullOuter -(37) CometProject +(35) CometProject Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#21 END AS item_sk#24, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#25, cume_sales#12 AS web_sales#26, cume_sales#23 AS store_sales#27] -(38) CometExchange +(36) CometExchange Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(39) CometSort +(37) CometSort Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] -(40) CometColumnarToRow [codegen id : 7] +(38) CometWindowExec Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29], [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] -(41) Window -Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] - -(42) Filter [codegen id : 8] +(39) CometFilter Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) -(43) TakeOrderedAndProject +(40) CometTakeOrderedAndProject +Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#24 ASC NULLS FIRST,d_date#25 ASC NULLS FIRST], output=[item_sk#24,d_date#25,web_sales#26,store_sales#27,web_cumulative#28,store_cumulative#29]), [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29], 100, 0, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] + +(41) CometColumnarToRow [codegen id : 3] Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometProject (46) - +- CometFilter (45) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(45) CometFilter +(43) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(46) CometProject +(44) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(47) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(48) BroadcastExchange +(46) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt index c74adc3809..ff86e0f44c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt @@ -1,55 +1,53 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS _w0#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS _w0#6, ws_item_sk#1)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS _w0#12, ss_item_sk#7)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS _w0#12, ss_item_sk#7)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 47 eligible operators (74%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 42 out of 47 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt index dc6a7ea61f..493a61c885 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt @@ -1,68 +1,56 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (8) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (3) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (6) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #8 - WholeStageCodegen (4) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #6 +WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometWindowExec [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #2 + CometProject [item_sk,d_date,cume_sales] + CometWindowExec [item_sk,d_date,_w0,ws_item_sk,cume_sales] + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [ws_item_sk,d_date] #4 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #6 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #7 + CometProject [item_sk,d_date,cume_sales] + CometWindowExec [item_sk,d_date,_w0,ss_item_sk,cume_sales] + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometColumnarExchange [ss_item_sk] #8 + WholeStageCodegen (2) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [ss_item_sk,d_date] #9 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt index 474ac8078d..fbcbd86dc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) +* CometColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometProject (28) + +- CometFilter (27) + +- CometWindowExec (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColum Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] +(26) CometWindowExec Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +Arguments: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23], [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] -(27) Window -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] - -(28) Filter [codegen id : 3] +(27) CometFilter Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 3] -Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +(28) CometProject Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] +Arguments: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] + +(29) CometTakeOrderedAndProject +Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[avg_quarterly_sales#23 ASC NULLS FIRST,sum_sales#21 ASC NULLS FIRST,i_manufact_id#5 ASC NULLS FIRST], output=[i_manufact_id#5,sum_sales#21,avg_quarterly_sales#23]), [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23], 100, 0, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -(30) TakeOrderedAndProject +(30) CometColumnarToRow [codegen id : 2] Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt index 1e2988c4c9..496649c274 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] @@ -35,4 +35,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt index eab0efef6c..428cc26f34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt @@ -1,44 +1,42 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (3) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manufact_id,sum_sales,avg_quarterly_sales] + CometProject [i_manufact_id,sum_sales,avg_quarterly_sales] + CometFilter [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] + CometWindowExec [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] + CometSort [i_manufact_id,sum_sales,_w0] + CometColumnarExchange [i_manufact_id] #1 + WholeStageCodegen (1) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt index 474ac8078d..fbcbd86dc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) +* CometColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometProject (28) + +- CometFilter (27) + +- CometWindowExec (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColum Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] +(26) CometWindowExec Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +Arguments: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23], [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] -(27) Window -Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] - -(28) Filter [codegen id : 3] +(27) CometFilter Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 3] -Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +(28) CometProject Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] +Arguments: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] + +(29) CometTakeOrderedAndProject +Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[avg_quarterly_sales#23 ASC NULLS FIRST,sum_sales#21 ASC NULLS FIRST,i_manufact_id#5 ASC NULLS FIRST], output=[i_manufact_id#5,sum_sales#21,avg_quarterly_sales#23]), [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23], 100, 0, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -(30) TakeOrderedAndProject +(30) CometColumnarToRow [codegen id : 2] Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt index 1e2988c4c9..496649c274 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] @@ -35,4 +35,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt index eab0efef6c..428cc26f34 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt @@ -1,44 +1,42 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (3) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manufact_id,sum_sales,avg_quarterly_sales] + CometProject [i_manufact_id,sum_sales,avg_quarterly_sales] + CometFilter [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] + CometWindowExec [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] + CometSort [i_manufact_id,sum_sales,_w0] + CometColumnarExchange [i_manufact_id] #1 + WholeStageCodegen (1) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt index eb0ce14345..ae9b283481 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt @@ -1,54 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (29) + : : +- CometFilter (28) + : : +- CometWindowExec (27) + : : +- CometFilter (26) + : : +- CometWindowExec (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- CometBroadcastExchange (37) + : +- CometProject (36) + : +- CometWindowExec (35) + : +- CometSort (34) + : +- CometColumnarExchange (33) + : +- * HashAggregate (32) + : +- * CometColumnarToRow (31) + : +- ReusedExchange (30) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -166,136 +160,112 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 2] +(25) CometWindowExec Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(27) Filter [codegen id : 3] +(26) CometFilter Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(28) Window +(27) CometWindowExec Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(29) Filter [codegen id : 10] +(28) CometFilter Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(30) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +(29) CometProject Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(31) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(32) CometColumnarToRow [codegen id : 4] +(31) CometColumnarToRow [codegen id : 2] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(33) HashAggregate [codegen id : 4] +(32) HashAggregate [codegen id : 2] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] -(34) CometColumnarExchange +(33) CometColumnarExchange Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(35) CometSort +(34) CometSort Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] - -(37) Window +(35) CometWindowExec Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30], [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(36) CometProject Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] +Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30], [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(39) BroadcastExchange +(37) CometBroadcastExchange Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +Right output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)], Inner, BuildRight -(41) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] +(39) CometProject Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -(42) ReusedExchange [Reuses operator id: 34] -Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] - -(43) CometSort -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 8] -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] - -(45) Window -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(46) Project [codegen id : 9] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] +(40) ReusedExchange [Reuses operator id: 37] +Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -(47) BroadcastExchange -Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] +Right output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#31, i_brand#32, cc_name#33, (rn#35 - 1)], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 1)] -Join type: Inner -Join condition: None +(42) CometProject +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#36, sum_sales#34 AS nsum#37] -(49) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(43) CometTakeOrderedAndProject +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,cc_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,cc_name#15,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#36,nsum#37]), [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] -(50) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(44) CometColumnarToRow [codegen id : 4] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(53) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) BroadcastExchange +(48) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt index 8c7324af76..8d50c97859 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt @@ -1,80 +1,78 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] @@ -105,4 +103,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt index dc90429f65..6ee00f9e29 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt @@ -1,77 +1,57 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (10) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (3) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (2) +WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,cc_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] CometColumnarToRow InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt index eb0ce14345..ae9b283481 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt @@ -1,54 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (29) + : : +- CometFilter (28) + : : +- CometWindowExec (27) + : : +- CometFilter (26) + : : +- CometWindowExec (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- CometBroadcastExchange (37) + : +- CometProject (36) + : +- CometWindowExec (35) + : +- CometSort (34) + : +- CometColumnarExchange (33) + : +- * HashAggregate (32) + : +- * CometColumnarToRow (31) + : +- ReusedExchange (30) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -166,136 +160,112 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 2] +(25) CometWindowExec Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(27) Filter [codegen id : 3] +(26) CometFilter Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(28) Window +(27) CometWindowExec Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(29) Filter [codegen id : 10] +(28) CometFilter Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(30) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +(29) CometProject Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(31) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(32) CometColumnarToRow [codegen id : 4] +(31) CometColumnarToRow [codegen id : 2] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(33) HashAggregate [codegen id : 4] +(32) HashAggregate [codegen id : 2] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] -(34) CometColumnarExchange +(33) CometColumnarExchange Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(35) CometSort +(34) CometSort Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] - -(37) Window +(35) CometWindowExec Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30], [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(36) CometProject Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] +Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30], [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(39) BroadcastExchange +(37) CometBroadcastExchange Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +Right output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)], Inner, BuildRight -(41) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] +(39) CometProject Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -(42) ReusedExchange [Reuses operator id: 34] -Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] - -(43) CometSort -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 8] -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] - -(45) Window -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(46) Project [codegen id : 9] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] +(40) ReusedExchange [Reuses operator id: 37] +Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -(47) BroadcastExchange -Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] +Right output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#31, i_brand#32, cc_name#33, (rn#35 - 1)], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 1)] -Join type: Inner -Join condition: None +(42) CometProject +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#36, sum_sales#34 AS nsum#37] -(49) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(43) CometTakeOrderedAndProject +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,cc_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,cc_name#15,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#36,nsum#37]), [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] -(50) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(44) CometColumnarToRow [codegen id : 4] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(53) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) BroadcastExchange +(48) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt index 8c7324af76..8d50c97859 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt @@ -1,80 +1,78 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] @@ -105,4 +103,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt index dc90429f65..6ee00f9e29 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt @@ -1,77 +1,57 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (10) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (3) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (2) +WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,cc_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] CometColumnarToRow InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt index d927679b54..049b5567ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) +* CometColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometProject (28) + +- CometFilter (27) + +- CometWindowExec (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumn Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] +(26) CometWindowExec Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +Arguments: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23], [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] -(27) Window -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] - -(28) Filter [codegen id : 3] +(27) CometFilter Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 3] -Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +(28) CometProject Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] +Arguments: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] + +(29) CometTakeOrderedAndProject +Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_manager_id#5 ASC NULLS FIRST,avg_monthly_sales#23 ASC NULLS FIRST,sum_sales#21 ASC NULLS FIRST], output=[i_manager_id#5,sum_sales#21,avg_monthly_sales#23]), [i_manager_id#5, sum_sales#21, avg_monthly_sales#23], 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -(30) TakeOrderedAndProject +(30) CometColumnarToRow [codegen id : 2] Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt index 57a78c7461..a622e150da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manager_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] @@ -35,4 +35,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt index 6688a7a467..ca60ad63cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt @@ -1,44 +1,42 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (3) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manager_id,sum_sales,avg_monthly_sales] + CometProject [i_manager_id,sum_sales,avg_monthly_sales] + CometFilter [i_manager_id,sum_sales,_w0,avg_monthly_sales] + CometWindowExec [i_manager_id,sum_sales,_w0,avg_monthly_sales] + CometSort [i_manager_id,sum_sales,_w0] + CometColumnarExchange [i_manager_id] #1 + WholeStageCodegen (1) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_manager_id,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt index d927679b54..049b5567ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) +* CometColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometProject (28) + +- CometFilter (27) + +- CometWindowExec (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumn Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] +(26) CometWindowExec Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +Arguments: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23], [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] -(27) Window -Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] - -(28) Filter [codegen id : 3] +(27) CometFilter Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END -(29) Project [codegen id : 3] -Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +(28) CometProject Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] +Arguments: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] + +(29) CometTakeOrderedAndProject +Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_manager_id#5 ASC NULLS FIRST,avg_monthly_sales#23 ASC NULLS FIRST,sum_sales#21 ASC NULLS FIRST], output=[i_manager_id#5,sum_sales#21,avg_monthly_sales#23]), [i_manager_id#5, sum_sales#21, avg_monthly_sales#23], 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -(30) TakeOrderedAndProject +(30) CometColumnarToRow [codegen id : 2] Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt index 57a78c7461..a622e150da 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manager_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] @@ -35,4 +35,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt index 6688a7a467..ca60ad63cc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt @@ -1,44 +1,42 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (3) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manager_id,sum_sales,avg_monthly_sales] + CometProject [i_manager_id,sum_sales,avg_monthly_sales] + CometFilter [i_manager_id,sum_sales,_w0,avg_monthly_sales] + CometWindowExec [i_manager_id,sum_sales,_w0,avg_monthly_sales] + CometSort [i_manager_id,sum_sales,_w0] + CometColumnarExchange [i_manager_id] #1 + WholeStageCodegen (1) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_manager_id,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt index 23cfae9635..b2d0cf6078 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- Window (47) - +- * CometColumnarToRow (46) +* CometColumnarToRow (49) ++- CometTakeOrderedAndProject (48) + +- CometProject (47) + +- CometWindowExec (46) +- CometSort (45) +- CometColumnarExchange (44) +- * HashAggregate (43) @@ -259,20 +259,20 @@ Arguments: hashpartitioning(_w1#32, _w2#33, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33], [_w1#32 ASC NULLS FIRST, _w2#33 ASC NULLS FIRST, _w0#31 DESC NULLS LAST] -(46) CometColumnarToRow [codegen id : 6] +(46) CometWindowExec Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] +Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34], [rank(_w0#31) windowspecdefinition(_w1#32, _w2#33, _w0#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [_w1#32, _w2#33], [_w0#31 DESC NULLS LAST] -(47) Window -Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] -Arguments: [rank(_w0#31) windowspecdefinition(_w1#32, _w2#33, _w0#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [_w1#32, _w2#33], [_w0#31 DESC NULLS LAST] - -(48) Project [codegen id : 7] -Output [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +(47) CometProject Input [8]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34] +Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] + +(48) CometTakeOrderedAndProject +Input [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#30 DESC NULLS LAST,CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST,rank_within_parent#34 ASC NULLS FIRST], output=[total_sum#29,s_state#23,s_county#24,lochierarchy#30,rank_within_parent#34]), [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34], 100, 0, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] -(49) TakeOrderedAndProject +(49) CometColumnarToRow [codegen id : 6] Input [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt index c07b33fdd7..3245d362b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate @@ -58,4 +58,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 53 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt index f6953d47ff..46e164b143 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt @@ -1,74 +1,72 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (7) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #4 +WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + CometColumnarToRow + InputAdapter + CometExchange [s_state] #7 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt index 23cfae9635..b2d0cf6078 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (49) -+- * Project (48) - +- Window (47) - +- * CometColumnarToRow (46) +* CometColumnarToRow (49) ++- CometTakeOrderedAndProject (48) + +- CometProject (47) + +- CometWindowExec (46) +- CometSort (45) +- CometColumnarExchange (44) +- * HashAggregate (43) @@ -259,20 +259,20 @@ Arguments: hashpartitioning(_w1#32, _w2#33, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33], [_w1#32 ASC NULLS FIRST, _w2#33 ASC NULLS FIRST, _w0#31 DESC NULLS LAST] -(46) CometColumnarToRow [codegen id : 6] +(46) CometWindowExec Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] +Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34], [rank(_w0#31) windowspecdefinition(_w1#32, _w2#33, _w0#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [_w1#32, _w2#33], [_w0#31 DESC NULLS LAST] -(47) Window -Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] -Arguments: [rank(_w0#31) windowspecdefinition(_w1#32, _w2#33, _w0#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [_w1#32, _w2#33], [_w0#31 DESC NULLS LAST] - -(48) Project [codegen id : 7] -Output [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +(47) CometProject Input [8]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34] +Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] + +(48) CometTakeOrderedAndProject +Input [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#30 DESC NULLS LAST,CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST,rank_within_parent#34 ASC NULLS FIRST], output=[total_sum#29,s_state#23,s_county#24,lochierarchy#30,rank_within_parent#34]), [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34], 100, 0, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] -(49) TakeOrderedAndProject +(49) CometColumnarToRow [codegen id : 6] Input [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt index c07b33fdd7..3245d362b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate @@ -58,4 +58,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 32 out of 53 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt index f6953d47ff..46e164b143 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt @@ -1,74 +1,72 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (7) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #4 +WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + CometColumnarToRow + InputAdapter + CometExchange [s_state] #7 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt index de788a42e2..f07df4bf44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * Project (24) - +- Window (23) - +- * CometColumnarToRow (22) +* CometColumnarToRow (25) ++- CometTakeOrderedAndProject (24) + +- CometProject (23) + +- CometWindowExec (22) +- CometSort (21) +- CometColumnarExchange (20) +- * HashAggregate (19) @@ -125,20 +125,20 @@ Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(22) CometColumnarToRow [codegen id : 2] +(22) CometWindowExec Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] +Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22], [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] -(23) Window -Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] - -(24) Project [codegen id : 3] -Output [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +(23) CometProject Input [8]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] +Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] + +(24) CometTakeOrderedAndProject +Input [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#18 DESC NULLS LAST,CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST,rank_within_parent#22 ASC NULLS FIRST], output=[total_sum#17,i_category#12,i_class#13,lochierarchy#18,rank_within_parent#22]), [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22], 100, 0, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] -(25) TakeOrderedAndProject +(25) CometColumnarToRow [codegen id : 2] Input [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] -Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt index 04dea63cb7..48e02a36c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS total_sum#3, i_category#4, i_class#5, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS lochierarchy#7, MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS _w0#8, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS _w1#9, CASE WHEN (cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint) = 0) THEN i_category#4 END AS _w2#10)] @@ -30,4 +30,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 22 out of 28 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt index 539b436013..98ab180c02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt @@ -1,39 +1,37 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (3) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] - CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt index de788a42e2..f07df4bf44 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (25) -+- * Project (24) - +- Window (23) - +- * CometColumnarToRow (22) +* CometColumnarToRow (25) ++- CometTakeOrderedAndProject (24) + +- CometProject (23) + +- CometWindowExec (22) +- CometSort (21) +- CometColumnarExchange (20) +- * HashAggregate (19) @@ -125,20 +125,20 @@ Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(22) CometColumnarToRow [codegen id : 2] +(22) CometWindowExec Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] +Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22], [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] -(23) Window -Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] - -(24) Project [codegen id : 3] -Output [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +(23) CometProject Input [8]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] +Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] + +(24) CometTakeOrderedAndProject +Input [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#18 DESC NULLS LAST,CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST,rank_within_parent#22 ASC NULLS FIRST], output=[total_sum#17,i_category#12,i_class#13,lochierarchy#18,rank_within_parent#22]), [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22], 100, 0, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] -(25) TakeOrderedAndProject +(25) CometColumnarToRow [codegen id : 2] Input [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] -Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt index 04dea63cb7..48e02a36c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS total_sum#3, i_category#4, i_class#5, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS lochierarchy#7, MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS _w0#8, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS _w1#9, CASE WHEN (cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint) = 0) THEN i_category#4 END AS _w2#10)] @@ -30,4 +30,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 22 out of 28 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt index 539b436013..98ab180c02 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt @@ -1,39 +1,37 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (3) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] - CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt index 974bebdf5e..75d2f28ba6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) +* CometColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometProject (28) + +- CometFilter (27) + +- CometWindowExec (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_ Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] +(26) CometWindowExec Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] +Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23], [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] -(27) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] - -(28) Filter [codegen id : 3] +(27) CometFilter Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END -(29) Project [codegen id : 3] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] +(28) CometProject Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] +Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] + +(29) CometTakeOrderedAndProject +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST,s_store_name#17 ASC NULLS FIRST], output=[i_category#7,i_class#6,i_brand#5,s_store_name#17,s_company_name#18,d_moy#15,sum_sales#21,avg_monthly_sales#23]), [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23], 100, 0, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -(30) TakeOrderedAndProject +(30) CometColumnarToRow [codegen id : 2] Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt index b206edf56c..224e191e1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_class#2, i_brand#3, s_store_name#4, s_company_name#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] @@ -35,4 +35,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt index 4fc9029de0..5c61e618cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt @@ -1,44 +1,42 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (3) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + CometProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + CometFilter [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] + CometWindowExec [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt index 974bebdf5e..75d2f28ba6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- * Filter (28) - +- Window (27) - +- * CometColumnarToRow (26) +* CometColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometProject (28) + +- CometFilter (27) + +- CometWindowExec (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_ Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 2] +(26) CometWindowExec Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] +Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23], [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] -(27) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] -Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] - -(28) Filter [codegen id : 3] +(27) CometFilter Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END -(29) Project [codegen id : 3] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] +(28) CometProject Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] +Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] + +(29) CometTakeOrderedAndProject +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST,s_store_name#17 ASC NULLS FIRST], output=[i_category#7,i_class#6,i_brand#5,s_store_name#17,s_company_name#18,d_moy#15,sum_sales#21,avg_monthly_sales#23]), [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23], 100, 0, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -(30) TakeOrderedAndProject +(30) CometColumnarToRow [codegen id : 2] Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt index b206edf56c..224e191e1d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_class#2, i_brand#3, s_store_name#4, s_company_name#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] @@ -35,4 +35,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt index 4fc9029de0..5c61e618cb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt @@ -1,44 +1,42 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (3) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + CometProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + CometFilter [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] + CometWindowExec [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt index 3c2b54e5b2..71782c1e0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt @@ -1,31 +1,30 @@ == Physical Plan == -* CometColumnarToRow (27) -+- CometProject (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (26) ++- CometProject (25) + +- CometSort (24) + +- CometExchange (23) + +- CometProject (22) + +- CometWindowExec (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -123,61 +122,58 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] +(22) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(24) CometColumnarExchange +(23) CometExchange Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(25) CometSort +(24) CometSort Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(26) CometProject +(25) CometProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(27) CometColumnarToRow [codegen id : 4] +(26) CometColumnarToRow [codegen id : 2] Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (32) -+- * CometColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (28) +BroadcastExchange (31) ++- * CometColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(29) CometFilter +(28) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(30) CometProject +(29) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(31) CometColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(32) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt index 85ea83a90b..c5e17e1883 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt @@ -1,35 +1,34 @@ CometColumnarToRow +- CometProject +- CometSort - +- CometColumnarExchange - +- Project - +- Window - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 29 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt index 3c91801f00..770e245610 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt @@ -1,43 +1,38 @@ -WholeStageCodegen (4) +WholeStageCodegen (2) CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (3) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt index 3c2b54e5b2..71782c1e0a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt @@ -1,31 +1,30 @@ == Physical Plan == -* CometColumnarToRow (27) -+- CometProject (26) - +- CometSort (25) - +- CometColumnarExchange (24) - +- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (26) ++- CometProject (25) + +- CometSort (24) + +- CometExchange (23) + +- CometProject (22) + +- CometWindowExec (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -123,61 +122,58 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] +(22) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(24) CometColumnarExchange +(23) CometExchange Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(25) CometSort +(24) CometSort Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(26) CometProject +(25) CometProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(27) CometColumnarToRow [codegen id : 4] +(26) CometColumnarToRow [codegen id : 2] Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (32) -+- * CometColumnarToRow (31) - +- CometProject (30) - +- CometFilter (29) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (28) +BroadcastExchange (31) ++- * CometColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(29) CometFilter +(28) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(30) CometProject +(29) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(31) CometColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(32) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt index 85ea83a90b..c5e17e1883 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt @@ -1,35 +1,34 @@ CometColumnarToRow +- CometProject +- CometSort - +- CometColumnarExchange - +- Project - +- Window - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 29 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt index 3c91801f00..770e245610 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt @@ -1,43 +1,38 @@ -WholeStageCodegen (4) +WholeStageCodegen (2) CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (3) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt index 93308784a9..a207bd8844 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) +* CometColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometProject (21) + +- CometWindowExec (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +(21) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(23) TakeOrderedAndProject +(22) CometTakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] + +(23) CometColumnarToRow [codegen id : 1] +Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt index 2708ea9eae..7b9553ba1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt index fb83fd2f9a..fbfb52828a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt @@ -1,35 +1,33 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index 93308784a9..a207bd8844 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) +* CometColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometProject (21) + +- CometWindowExec (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +(21) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(23) TakeOrderedAndProject +(22) CometTakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] + +(23) CometColumnarToRow [codegen id : 1] +Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt index 2708ea9eae..7b9553ba1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index fb83fd2f9a..fbfb52828a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -1,35 +1,33 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt index c23383e09a..1d8693e4aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) +* CometColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometProject (21) + +- CometWindowExec (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +(21) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(23) TakeOrderedAndProject +(22) CometTakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] + +(23) CometColumnarToRow [codegen id : 1] +Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt index b08f4f6d0e..3bd3bd46f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt index 2958d060fe..2d72ca58d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt @@ -1,35 +1,33 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index c23383e09a..1d8693e4aa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) +* CometColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometProject (21) + +- CometWindowExec (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +(21) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(23) TakeOrderedAndProject +(22) CometTakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] + +(23) CometColumnarToRow [codegen id : 1] +Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt index b08f4f6d0e..3bd3bd46f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index 2958d060fe..2d72ca58d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -1,35 +1,33 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt index ae1d4c242b..97885ed563 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- Window (28) - +- * CometColumnarToRow (27) +* CometColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometProject (28) + +- CometWindowExec (27) +- CometSort (26) +- CometExchange (25) +- CometHashAggregate (24) @@ -153,20 +153,20 @@ Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNative Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] -(27) CometColumnarToRow [codegen id : 1] +(27) CometWindowExec Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26], [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] -(28) Window -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] - -(29) Project [codegen id : 2] -Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +(28) CometProject Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] +Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] + +(29) CometTakeOrderedAndProject +Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#22 DESC NULLS LAST,CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST,rank_within_parent#26 ASC NULLS FIRST], output=[gross_margin#21,i_category#16,i_class#17,lochierarchy#22,rank_within_parent#26]), [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26], 100, 0, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -(30) TakeOrderedAndProject +(30) CometColumnarToRow [codegen id : 1] Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt index 325f54c02f..0f5f924dd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -35,4 +35,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 34 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt index 61782e3011..b6a5cf7024 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt @@ -1,42 +1,40 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] - CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt index ae1d4c242b..97885ed563 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Project (29) - +- Window (28) - +- * CometColumnarToRow (27) +* CometColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometProject (28) + +- CometWindowExec (27) +- CometSort (26) +- CometExchange (25) +- CometHashAggregate (24) @@ -153,20 +153,20 @@ Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNative Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] -(27) CometColumnarToRow [codegen id : 1] +(27) CometWindowExec Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26], [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] -(28) Window -Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] - -(29) Project [codegen id : 2] -Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +(28) CometProject Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] +Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] + +(29) CometTakeOrderedAndProject +Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#22 DESC NULLS LAST,CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST,rank_within_parent#26 ASC NULLS FIRST], output=[gross_margin#21,i_category#16,i_class#17,lochierarchy#22,rank_within_parent#26]), [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26], 100, 0, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -(30) TakeOrderedAndProject +(30) CometColumnarToRow [codegen id : 1] Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt index 325f54c02f..0f5f924dd6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -35,4 +35,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 34 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index 61782e3011..b6a5cf7024 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -1,42 +1,40 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] - CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt index 134ab7939e..bb7bf636c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt @@ -1,38 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (23) - : : +- * SortMergeJoin Inner (22) - : : :- * Sort (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- Window (11) - : : : +- * CometColumnarToRow (10) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- * Sort (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- Window (18) - : : +- * CometColumnarToRow (17) - : : +- CometSort (16) - : : +- ReusedExchange (15) - : +- BroadcastExchange (28) - : +- * CometColumnarToRow (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - +- ReusedExchange (31) +* CometColumnarToRow (32) ++- CometTakeOrderedAndProject (31) + +- CometProject (30) + +- CometBroadcastHashJoin (29) + :- CometProject (27) + : +- CometBroadcastHashJoin (26) + : :- CometProject (21) + : : +- CometSortMergeJoin (20) + : : :- CometSort (13) + : : : +- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometWindowExec (10) + : : : +- CometSort (9) + : : : +- CometExchange (8) + : : : +- CometFilter (7) + : : : +- CometHashAggregate (6) + : : : +- CometExchange (5) + : : : +- CometHashAggregate (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- CometSort (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometWindowExec (16) + : : +- CometSort (15) + : : +- ReusedExchange (14) + : +- CometBroadcastExchange (25) + : +- CometProject (24) + : +- CometFilter (23) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) + +- ReusedExchange (28) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -76,152 +74,143 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(10) CometColumnarToRow [codegen id : 1] +(10) CometWindowExec Input [2]: [item_sk#7, rank_col#8] +Arguments: [item_sk#7, rank_col#8, rnk#11], [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] -(11) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(12) Filter [codegen id : 2] +(11) CometFilter Input [3]: [item_sk#7, rank_col#8, rnk#11] Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) -(13) Project [codegen id : 2] -Output [2]: [item_sk#7, rnk#11] +(12) CometProject Input [3]: [item_sk#7, rank_col#8, rnk#11] +Arguments: [item_sk#7, rnk#11], [item_sk#7, rnk#11] -(14) Sort [codegen id : 2] +(13) CometSort Input [2]: [item_sk#7, rnk#11] -Arguments: [rnk#11 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#7, rnk#11], [rnk#11 ASC NULLS FIRST] -(15) ReusedExchange [Reuses operator id: 8] +(14) ReusedExchange [Reuses operator id: 8] Output [2]: [item_sk#12, rank_col#13] -(16) CometSort +(15) CometSort Input [2]: [item_sk#12, rank_col#13] Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] -(17) CometColumnarToRow [codegen id : 3] +(16) CometWindowExec Input [2]: [item_sk#12, rank_col#13] +Arguments: [item_sk#12, rank_col#13, rnk#14], [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] -(18) Window -Input [2]: [item_sk#12, rank_col#13] -Arguments: [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] - -(19) Filter [codegen id : 4] +(17) CometFilter Input [3]: [item_sk#12, rank_col#13, rnk#14] Condition : ((rnk#14 < 11) AND isnotnull(item_sk#12)) -(20) Project [codegen id : 4] -Output [2]: [item_sk#12, rnk#14] +(18) CometProject Input [3]: [item_sk#12, rank_col#13, rnk#14] +Arguments: [item_sk#12, rnk#14], [item_sk#12, rnk#14] -(21) Sort [codegen id : 4] +(19) CometSort Input [2]: [item_sk#12, rnk#14] -Arguments: [rnk#14 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#12, rnk#14], [rnk#14 ASC NULLS FIRST] -(22) SortMergeJoin [codegen id : 7] -Left keys [1]: [rnk#11] -Right keys [1]: [rnk#14] -Join type: Inner -Join condition: None +(20) CometSortMergeJoin +Left output [2]: [item_sk#7, rnk#11] +Right output [2]: [item_sk#12, rnk#14] +Arguments: [rnk#11], [rnk#14], Inner -(23) Project [codegen id : 7] -Output [3]: [item_sk#7, rnk#11, item_sk#12] +(21) CometProject Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] +Arguments: [item_sk#7, rnk#11, item_sk#12], [item_sk#7, rnk#11, item_sk#12] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [2]: [i_item_sk#15, i_product_name#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(25) CometFilter +(23) CometFilter Input [2]: [i_item_sk#15, i_product_name#16] Condition : isnotnull(i_item_sk#15) -(26) CometProject +(24) CometProject Input [2]: [i_item_sk#15, i_product_name#16] Arguments: [i_item_sk#15, i_product_name#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#16, 50, true, false, true) AS i_product_name#17] -(27) CometColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#15, i_product_name#17] - -(28) BroadcastExchange +(25) CometBroadcastExchange Input [2]: [i_item_sk#15, i_product_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#15, i_product_name#17] -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None +(26) CometBroadcastHashJoin +Left output [3]: [item_sk#7, rnk#11, item_sk#12] +Right output [2]: [i_item_sk#15, i_product_name#17] +Arguments: [item_sk#7], [i_item_sk#15], Inner, BuildRight -(30) Project [codegen id : 7] -Output [3]: [rnk#11, item_sk#12, i_product_name#17] +(27) CometProject Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#17] +Arguments: [rnk#11, item_sk#12, i_product_name#17], [rnk#11, item_sk#12, i_product_name#17] -(31) ReusedExchange [Reuses operator id: 28] +(28) ReusedExchange [Reuses operator id: 25] Output [2]: [i_item_sk#18, i_product_name#19] -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#12] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None +(29) CometBroadcastHashJoin +Left output [3]: [rnk#11, item_sk#12, i_product_name#17] +Right output [2]: [i_item_sk#18, i_product_name#19] +Arguments: [item_sk#12], [i_item_sk#18], Inner, BuildRight -(33) Project [codegen id : 7] -Output [3]: [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] +(30) CometProject Input [5]: [rnk#11, item_sk#12, i_product_name#17, i_item_sk#18, i_product_name#19] +Arguments: [rnk#11, best_performing#20, worst_performing#21], [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] + +(31) CometTakeOrderedAndProject +Input [3]: [rnk#11, best_performing#20, worst_performing#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[rnk#11 ASC NULLS FIRST], output=[rnk#11,best_performing#20,worst_performing#21]), [rnk#11, best_performing#20, worst_performing#21], 100, 0, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_performing#21] -(34) TakeOrderedAndProject +(32) CometColumnarToRow [codegen id : 1] Input [3]: [rnk#11, best_performing#20, worst_performing#21] -Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_performing#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (35) +* CometColumnarToRow (39) ++- CometHashAggregate (38) + +- CometExchange (37) + +- CometHashAggregate (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (33) -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(36) CometFilter +(34) CometFilter Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Condition : ((isnotnull(ss_store_sk#23) AND (ss_store_sk#23 = 4)) AND isnull(ss_addr_sk#22)) -(37) CometProject +(35) CometProject Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Arguments: [ss_store_sk#23, ss_net_profit#24], [ss_store_sk#23, ss_net_profit#24] -(38) CometHashAggregate +(36) CometHashAggregate Input [2]: [ss_store_sk#23, ss_net_profit#24] Keys [1]: [ss_store_sk#23] Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#24))] -(39) CometExchange +(37) CometExchange Input [3]: [ss_store_sk#23, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) CometHashAggregate +(38) CometHashAggregate Input [3]: [ss_store_sk#23, sum#26, count#27] Keys [1]: [ss_store_sk#23] Functions [1]: [avg(UnscaledValue(ss_net_profit#24))] -(41) CometColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [1]: [rank_col#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt index 9a215099d4..44ae94ee91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt @@ -1,63 +1,60 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- SortMergeJoin - : : :- Sort - : : : +- Project - : : : +- Filter - : : : +- Window - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- Sort - : : +- Project - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometSortMergeJoin + : : :- CometSort + : : : +- CometProject + : : : +- CometFilter + : : : +- CometWindowExec + : : : +- CometSort + : : : +- CometExchange + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- CometSort + : : +- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 36 out of 55 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 53 out of 55 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt index e3ec416397..705ece31b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt @@ -1,59 +1,44 @@ -TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (7) - Project [rnk,i_product_name,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [rnk,item_sk,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [item_sk,rnk,item_sk] - SortMergeJoin [rnk,rnk] - InputAdapter - WholeStageCodegen (2) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometExchange #1 - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - ReusedExchange [item_sk,rank_col] #1 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #4 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [rnk,best_performing,worst_performing] + CometProject [i_product_name,i_product_name] [rnk,best_performing,worst_performing] + CometBroadcastHashJoin [rnk,item_sk,i_product_name,i_item_sk,i_product_name] + CometProject [rnk,item_sk,i_product_name] + CometBroadcastHashJoin [item_sk,rnk,item_sk,i_item_sk,i_product_name] + CometProject [item_sk,rnk,item_sk] + CometSortMergeJoin [item_sk,rnk,item_sk,rnk] + CometSort [item_sk,rnk] + CometProject [item_sk,rnk] + CometFilter [item_sk,rank_col,rnk] + CometWindowExec [item_sk,rank_col,rnk] + CometSort [item_sk,rank_col] + CometExchange #1 + CometFilter [item_sk,rank_col] + Subquery #1 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_store_sk] #3 + CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] + CometProject [ss_store_sk,ss_net_profit] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_item_sk] #2 + CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] + CometProject [ss_item_sk,ss_net_profit] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometSort [item_sk,rnk] + CometProject [item_sk,rnk] + CometFilter [item_sk,rank_col,rnk] + CometWindowExec [item_sk,rank_col,rnk] + CometSort [item_sk,rank_col] + ReusedExchange [item_sk,rank_col] #1 + CometBroadcastExchange [i_item_sk,i_product_name] #4 + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] + ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index 134ab7939e..bb7bf636c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -1,38 +1,36 @@ == Physical Plan == -TakeOrderedAndProject (34) -+- * Project (33) - +- * BroadcastHashJoin Inner BuildRight (32) - :- * Project (30) - : +- * BroadcastHashJoin Inner BuildRight (29) - : :- * Project (23) - : : +- * SortMergeJoin Inner (22) - : : :- * Sort (14) - : : : +- * Project (13) - : : : +- * Filter (12) - : : : +- Window (11) - : : : +- * CometColumnarToRow (10) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- * Sort (21) - : : +- * Project (20) - : : +- * Filter (19) - : : +- Window (18) - : : +- * CometColumnarToRow (17) - : : +- CometSort (16) - : : +- ReusedExchange (15) - : +- BroadcastExchange (28) - : +- * CometColumnarToRow (27) - : +- CometProject (26) - : +- CometFilter (25) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - +- ReusedExchange (31) +* CometColumnarToRow (32) ++- CometTakeOrderedAndProject (31) + +- CometProject (30) + +- CometBroadcastHashJoin (29) + :- CometProject (27) + : +- CometBroadcastHashJoin (26) + : :- CometProject (21) + : : +- CometSortMergeJoin (20) + : : :- CometSort (13) + : : : +- CometProject (12) + : : : +- CometFilter (11) + : : : +- CometWindowExec (10) + : : : +- CometSort (9) + : : : +- CometExchange (8) + : : : +- CometFilter (7) + : : : +- CometHashAggregate (6) + : : : +- CometExchange (5) + : : : +- CometHashAggregate (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- CometSort (19) + : : +- CometProject (18) + : : +- CometFilter (17) + : : +- CometWindowExec (16) + : : +- CometSort (15) + : : +- ReusedExchange (14) + : +- CometBroadcastExchange (25) + : +- CometProject (24) + : +- CometFilter (23) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) + +- ReusedExchange (28) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -76,152 +74,143 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(10) CometColumnarToRow [codegen id : 1] +(10) CometWindowExec Input [2]: [item_sk#7, rank_col#8] +Arguments: [item_sk#7, rank_col#8, rnk#11], [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] -(11) Window -Input [2]: [item_sk#7, rank_col#8] -Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] - -(12) Filter [codegen id : 2] +(11) CometFilter Input [3]: [item_sk#7, rank_col#8, rnk#11] Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) -(13) Project [codegen id : 2] -Output [2]: [item_sk#7, rnk#11] +(12) CometProject Input [3]: [item_sk#7, rank_col#8, rnk#11] +Arguments: [item_sk#7, rnk#11], [item_sk#7, rnk#11] -(14) Sort [codegen id : 2] +(13) CometSort Input [2]: [item_sk#7, rnk#11] -Arguments: [rnk#11 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#7, rnk#11], [rnk#11 ASC NULLS FIRST] -(15) ReusedExchange [Reuses operator id: 8] +(14) ReusedExchange [Reuses operator id: 8] Output [2]: [item_sk#12, rank_col#13] -(16) CometSort +(15) CometSort Input [2]: [item_sk#12, rank_col#13] Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] -(17) CometColumnarToRow [codegen id : 3] +(16) CometWindowExec Input [2]: [item_sk#12, rank_col#13] +Arguments: [item_sk#12, rank_col#13, rnk#14], [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] -(18) Window -Input [2]: [item_sk#12, rank_col#13] -Arguments: [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] - -(19) Filter [codegen id : 4] +(17) CometFilter Input [3]: [item_sk#12, rank_col#13, rnk#14] Condition : ((rnk#14 < 11) AND isnotnull(item_sk#12)) -(20) Project [codegen id : 4] -Output [2]: [item_sk#12, rnk#14] +(18) CometProject Input [3]: [item_sk#12, rank_col#13, rnk#14] +Arguments: [item_sk#12, rnk#14], [item_sk#12, rnk#14] -(21) Sort [codegen id : 4] +(19) CometSort Input [2]: [item_sk#12, rnk#14] -Arguments: [rnk#14 ASC NULLS FIRST], false, 0 +Arguments: [item_sk#12, rnk#14], [rnk#14 ASC NULLS FIRST] -(22) SortMergeJoin [codegen id : 7] -Left keys [1]: [rnk#11] -Right keys [1]: [rnk#14] -Join type: Inner -Join condition: None +(20) CometSortMergeJoin +Left output [2]: [item_sk#7, rnk#11] +Right output [2]: [item_sk#12, rnk#14] +Arguments: [rnk#11], [rnk#14], Inner -(23) Project [codegen id : 7] -Output [3]: [item_sk#7, rnk#11, item_sk#12] +(21) CometProject Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] +Arguments: [item_sk#7, rnk#11, item_sk#12], [item_sk#7, rnk#11, item_sk#12] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [2]: [i_item_sk#15, i_product_name#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(25) CometFilter +(23) CometFilter Input [2]: [i_item_sk#15, i_product_name#16] Condition : isnotnull(i_item_sk#15) -(26) CometProject +(24) CometProject Input [2]: [i_item_sk#15, i_product_name#16] Arguments: [i_item_sk#15, i_product_name#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#16, 50, true, false, true) AS i_product_name#17] -(27) CometColumnarToRow [codegen id : 5] -Input [2]: [i_item_sk#15, i_product_name#17] - -(28) BroadcastExchange +(25) CometBroadcastExchange Input [2]: [i_item_sk#15, i_product_name#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [i_item_sk#15, i_product_name#17] -(29) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#7] -Right keys [1]: [i_item_sk#15] -Join type: Inner -Join condition: None +(26) CometBroadcastHashJoin +Left output [3]: [item_sk#7, rnk#11, item_sk#12] +Right output [2]: [i_item_sk#15, i_product_name#17] +Arguments: [item_sk#7], [i_item_sk#15], Inner, BuildRight -(30) Project [codegen id : 7] -Output [3]: [rnk#11, item_sk#12, i_product_name#17] +(27) CometProject Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#17] +Arguments: [rnk#11, item_sk#12, i_product_name#17], [rnk#11, item_sk#12, i_product_name#17] -(31) ReusedExchange [Reuses operator id: 28] +(28) ReusedExchange [Reuses operator id: 25] Output [2]: [i_item_sk#18, i_product_name#19] -(32) BroadcastHashJoin [codegen id : 7] -Left keys [1]: [item_sk#12] -Right keys [1]: [i_item_sk#18] -Join type: Inner -Join condition: None +(29) CometBroadcastHashJoin +Left output [3]: [rnk#11, item_sk#12, i_product_name#17] +Right output [2]: [i_item_sk#18, i_product_name#19] +Arguments: [item_sk#12], [i_item_sk#18], Inner, BuildRight -(33) Project [codegen id : 7] -Output [3]: [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] +(30) CometProject Input [5]: [rnk#11, item_sk#12, i_product_name#17, i_item_sk#18, i_product_name#19] +Arguments: [rnk#11, best_performing#20, worst_performing#21], [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] + +(31) CometTakeOrderedAndProject +Input [3]: [rnk#11, best_performing#20, worst_performing#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[rnk#11 ASC NULLS FIRST], output=[rnk#11,best_performing#20,worst_performing#21]), [rnk#11, best_performing#20, worst_performing#21], 100, 0, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_performing#21] -(34) TakeOrderedAndProject +(32) CometColumnarToRow [codegen id : 1] Input [3]: [rnk#11, best_performing#20, worst_performing#21] -Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_performing#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (41) -+- CometHashAggregate (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometProject (37) - +- CometFilter (36) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (35) +* CometColumnarToRow (39) ++- CometHashAggregate (38) + +- CometExchange (37) + +- CometHashAggregate (36) + +- CometProject (35) + +- CometFilter (34) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (33) -(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(36) CometFilter +(34) CometFilter Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Condition : ((isnotnull(ss_store_sk#23) AND (ss_store_sk#23 = 4)) AND isnull(ss_addr_sk#22)) -(37) CometProject +(35) CometProject Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Arguments: [ss_store_sk#23, ss_net_profit#24], [ss_store_sk#23, ss_net_profit#24] -(38) CometHashAggregate +(36) CometHashAggregate Input [2]: [ss_store_sk#23, ss_net_profit#24] Keys [1]: [ss_store_sk#23] Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#24))] -(39) CometExchange +(37) CometExchange Input [3]: [ss_store_sk#23, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) CometHashAggregate +(38) CometHashAggregate Input [3]: [ss_store_sk#23, sum#26, count#27] Keys [1]: [ss_store_sk#23] Functions [1]: [avg(UnscaledValue(ss_net_profit#24))] -(41) CometColumnarToRow [codegen id : 1] +(39) CometColumnarToRow [codegen id : 1] Input [1]: [rank_col#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt index 9a215099d4..44ae94ee91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt @@ -1,63 +1,60 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- SortMergeJoin - : : :- Sort - : : : +- Project - : : : +- Filter - : : : +- Window - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- Sort - : : +- Project - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- BroadcastExchange - : +- CometColumnarToRow - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- BroadcastExchange - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometSortMergeJoin + : : :- CometSort + : : : +- CometProject + : : : +- CometFilter + : : : +- CometWindowExec + : : : +- CometSort + : : : +- CometExchange + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- CometSort + : : +- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 36 out of 55 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 53 out of 55 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index e3ec416397..705ece31b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -1,59 +1,44 @@ -TakeOrderedAndProject [rnk,best_performing,worst_performing] - WholeStageCodegen (7) - Project [rnk,i_product_name,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [rnk,item_sk,i_product_name] - BroadcastHashJoin [item_sk,i_item_sk] - Project [item_sk,rnk,item_sk] - SortMergeJoin [rnk,rnk] - InputAdapter - WholeStageCodegen (2) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - CometExchange #1 - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - InputAdapter - WholeStageCodegen (4) - Sort [rnk] - Project [item_sk,rnk] - Filter [rnk,item_sk] - InputAdapter - Window [rank_col] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,rank_col] - ReusedExchange [item_sk,rank_col] #1 - InputAdapter - BroadcastExchange #4 - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] - InputAdapter - ReusedExchange [i_item_sk,i_product_name] #4 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [rnk,best_performing,worst_performing] + CometProject [i_product_name,i_product_name] [rnk,best_performing,worst_performing] + CometBroadcastHashJoin [rnk,item_sk,i_product_name,i_item_sk,i_product_name] + CometProject [rnk,item_sk,i_product_name] + CometBroadcastHashJoin [item_sk,rnk,item_sk,i_item_sk,i_product_name] + CometProject [item_sk,rnk,item_sk] + CometSortMergeJoin [item_sk,rnk,item_sk,rnk] + CometSort [item_sk,rnk] + CometProject [item_sk,rnk] + CometFilter [item_sk,rank_col,rnk] + CometWindowExec [item_sk,rank_col,rnk] + CometSort [item_sk,rank_col] + CometExchange #1 + CometFilter [item_sk,rank_col] + Subquery #1 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_store_sk] #3 + CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] + CometProject [ss_store_sk,ss_net_profit] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_item_sk] #2 + CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] + CometProject [ss_item_sk,ss_net_profit] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometSort [item_sk,rnk] + CometProject [item_sk,rnk] + CometFilter [item_sk,rank_col,rnk] + CometWindowExec [item_sk,rank_col,rnk] + CometSort [item_sk,rank_col] + ReusedExchange [item_sk,rank_col] #1 + CometBroadcastExchange [i_item_sk,i_product_name] #4 + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] + ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt index 1c039203ce..d3e2a1a2d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt @@ -1,52 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (28) + : : +- CometFilter (27) + : : +- CometWindowExec (26) + : : +- CometFilter (25) + : : +- CometWindowExec (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometWindowExec (33) + : +- CometSort (32) + : +- CometExchange (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + +- ReusedExchange (38) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -159,131 +153,107 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] +(24) CometWindowExec Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] +(25) CometFilter Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window +(26) CometWindowExec Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(28) Filter [codegen id : 7] +(27) CometFilter Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +(28) CometProject Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(30) ReusedExchange [Reuses operator id: 20] +(29) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -(31) CometHashAggregate +(30) CometHashAggregate Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] -(32) CometExchange +(31) CometExchange Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(33) CometSort +(32) CometSort Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window +(33) CometWindowExec Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28], [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] +(34) CometProject Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] +Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28], [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -(37) BroadcastExchange +(35) CometBroadcastExchange Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +Right output [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)], Inner, BuildRight -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +(37) CometProject Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] +(38) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +Right output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#37 - 1)], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None +(40) CometProject +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#38, sum_sales#36 AS nsum#39] -(47) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +(41) CometTakeOrderedAndProject +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,s_store_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,s_store_name#15,s_company_name#16,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#38,nsum#39]), [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -(48) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +(42) CometColumnarToRow [codegen id : 1] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(44) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(46) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt index 1bbdb7273f..8c361a8340 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt @@ -1,78 +1,76 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt index 19124a9e20..657e83912e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt @@ -1,71 +1,51 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index 1c039203ce..d3e2a1a2d6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -1,52 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (28) + : : +- CometFilter (27) + : : +- CometWindowExec (26) + : : +- CometFilter (25) + : : +- CometWindowExec (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometWindowExec (33) + : +- CometSort (32) + : +- CometExchange (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + +- ReusedExchange (38) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -159,131 +153,107 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] +(24) CometWindowExec Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] +(25) CometFilter Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window +(26) CometWindowExec Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(28) Filter [codegen id : 7] +(27) CometFilter Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +(28) CometProject Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(30) ReusedExchange [Reuses operator id: 20] +(29) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -(31) CometHashAggregate +(30) CometHashAggregate Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] -(32) CometExchange +(31) CometExchange Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(33) CometSort +(32) CometSort Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window +(33) CometWindowExec Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28], [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] +(34) CometProject Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] +Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28], [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -(37) BroadcastExchange +(35) CometBroadcastExchange Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +Right output [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)], Inner, BuildRight -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +(37) CometProject Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] +(38) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +Right output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#37 - 1)], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None +(40) CometProject +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#38, sum_sales#36 AS nsum#39] -(47) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +(41) CometTakeOrderedAndProject +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,s_store_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,s_store_name#15,s_company_name#16,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#38,nsum#39]), [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -(48) TakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +(42) CometColumnarToRow [codegen id : 1] +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(44) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(46) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt index 1bbdb7273f..8c361a8340 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt @@ -1,78 +1,76 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index 19124a9e20..657e83912e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -1,71 +1,51 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt index 338f5f4201..9c336d7e3c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt @@ -1,82 +1,79 @@ == Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) +* CometColumnarToRow (75) ++- CometTakeOrderedAndProject (74) + +- CometHashAggregate (73) + +- CometExchange (72) + +- CometHashAggregate (71) + +- CometUnion (70) + :- CometProject (25) + : +- CometFilter (24) + : +- CometWindowExec (23) + : +- CometSort (22) + : +- CometWindowExec (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- CometProject (47) + : +- CometFilter (46) + : +- CometWindowExec (45) + : +- CometSort (44) + : +- CometWindowExec (43) + : +- CometSort (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometExchange (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometBroadcastExchange (29) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (35) + +- CometProject (69) + +- CometFilter (68) + +- CometWindowExec (67) + +- CometSort (66) + +- CometWindowExec (65) + +- CometSort (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometProject (56) + : +- CometBroadcastHashJoin (55) + : :- CometBroadcastExchange (51) + : : +- CometProject (50) + : : +- CometFilter (49) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) + +- ReusedExchange (57) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -173,30 +170,27 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 1] +(21) CometWindowExec Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] +(22) CometSort Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] -(24) Window +(23) CometWindowExec Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(25) Filter [codegen id : 3] +(24) CometFilter Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(25) CometProject Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -204,100 +198,97 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(29) CometProject +(28) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometBroadcastExchange +(29) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(33) CometProject +(32) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(34) CometBroadcastHashJoin +(33) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(35) CometProject +(34) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(36) ReusedExchange [Reuses operator id: 13] +(35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(37) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(38) CometProject +(37) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(39) CometHashAggregate +(38) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(40) CometExchange +(39) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(41) CometHashAggregate +(40) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(42) CometExchange +(41) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort +(42) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window +(43) CometWindowExec Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(46) Sort [codegen id : 5] +(44) CometSort Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] -(47) Window +(45) CometWindowExec Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(48) Filter [codegen id : 6] +(46) CometFilter Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +(47) CometProject Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -305,158 +296,156 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter +(49) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(52) CometProject +(50) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(53) CometBroadcastExchange +(51) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(56) CometProject +(54) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(57) CometBroadcastHashJoin +(55) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(58) CometProject +(56) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(59) ReusedExchange [Reuses operator id: 13] +(57) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(60) CometBroadcastHashJoin +(58) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(61) CometProject +(59) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(62) CometHashAggregate +(60) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(61) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(64) CometHashAggregate +(62) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(65) CometExchange +(63) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) CometSort +(64) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) CometColumnarToRow [codegen id : 7] +(65) CometWindowExec Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] +(66) CometSort Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] -(70) Window +(67) CometWindowExec Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(71) Filter [codegen id : 9] +(68) CometFilter Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(69) CometProject Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(73) Union +(70) CometUnion +Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(74) HashAggregate [codegen id : 10] +(71) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarExchange +(72) CometExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(76) CometHashAggregate +(73) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(77) CometTakeOrderedAndProject +(74) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(78) CometColumnarToRow [codegen id : 11] +(75) CometColumnarToRow [codegen id : 1] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter +(77) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(81) CometProject +(78) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(79) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(83) BroadcastExchange +(80) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt index 4988a26564..6a8de7fe28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt @@ -1,94 +1,91 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt index ca80833ee5..4f128e83de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt @@ -1,110 +1,87 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometUnion [channel,item,return_ratio,return_rank,currency_rank] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 338f5f4201..9c336d7e3c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -1,82 +1,79 @@ == Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) +* CometColumnarToRow (75) ++- CometTakeOrderedAndProject (74) + +- CometHashAggregate (73) + +- CometExchange (72) + +- CometHashAggregate (71) + +- CometUnion (70) + :- CometProject (25) + : +- CometFilter (24) + : +- CometWindowExec (23) + : +- CometSort (22) + : +- CometWindowExec (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- CometProject (47) + : +- CometFilter (46) + : +- CometWindowExec (45) + : +- CometSort (44) + : +- CometWindowExec (43) + : +- CometSort (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometExchange (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometBroadcastExchange (29) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (35) + +- CometProject (69) + +- CometFilter (68) + +- CometWindowExec (67) + +- CometSort (66) + +- CometWindowExec (65) + +- CometSort (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometProject (56) + : +- CometBroadcastHashJoin (55) + : :- CometBroadcastExchange (51) + : : +- CometProject (50) + : : +- CometFilter (49) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) + +- ReusedExchange (57) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -173,30 +170,27 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 1] +(21) CometWindowExec Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] +(22) CometSort Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] -(24) Window +(23) CometWindowExec Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(25) Filter [codegen id : 3] +(24) CometFilter Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(25) CometProject Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -204,100 +198,97 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(29) CometProject +(28) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometBroadcastExchange +(29) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(33) CometProject +(32) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(34) CometBroadcastHashJoin +(33) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(35) CometProject +(34) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(36) ReusedExchange [Reuses operator id: 13] +(35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(37) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(38) CometProject +(37) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(39) CometHashAggregate +(38) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(40) CometExchange +(39) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(41) CometHashAggregate +(40) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(42) CometExchange +(41) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort +(42) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window +(43) CometWindowExec Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(46) Sort [codegen id : 5] +(44) CometSort Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] -(47) Window +(45) CometWindowExec Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(48) Filter [codegen id : 6] +(46) CometFilter Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +(47) CometProject Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -305,158 +296,156 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter +(49) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(52) CometProject +(50) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(53) CometBroadcastExchange +(51) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(56) CometProject +(54) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(57) CometBroadcastHashJoin +(55) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(58) CometProject +(56) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(59) ReusedExchange [Reuses operator id: 13] +(57) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(60) CometBroadcastHashJoin +(58) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(61) CometProject +(59) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(62) CometHashAggregate +(60) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(61) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(64) CometHashAggregate +(62) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(65) CometExchange +(63) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) CometSort +(64) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) CometColumnarToRow [codegen id : 7] +(65) CometWindowExec Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] +(66) CometSort Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] -(70) Window +(67) CometWindowExec Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(71) Filter [codegen id : 9] +(68) CometFilter Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(69) CometProject Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(73) Union +(70) CometUnion +Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(74) HashAggregate [codegen id : 10] +(71) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarExchange +(72) CometExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(76) CometHashAggregate +(73) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(77) CometTakeOrderedAndProject +(74) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(78) CometColumnarToRow [codegen id : 11] +(75) CometColumnarToRow [codegen id : 1] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter +(77) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(81) CometProject +(78) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(79) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(83) BroadcastExchange +(80) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt index 4988a26564..6a8de7fe28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt @@ -1,94 +1,91 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index ca80833ee5..4f128e83de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -1,110 +1,87 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometUnion [channel,item,return_ratio,return_rank,currency_rank] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt index 683cc25d24..4c7d196606 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt @@ -1,45 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Filter (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometProject (35) - +- CometSortMergeJoin (34) - :- CometSort (18) - : +- CometColumnarExchange (17) - : +- * Project (16) - : +- Window (15) - : +- * CometColumnarToRow (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (33) - +- CometColumnarExchange (32) - +- * Project (31) - +- Window (30) - +- * CometColumnarToRow (29) - +- CometSort (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) +* CometColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometFilter (37) + +- CometWindowExec (36) + +- CometSort (35) + +- CometExchange (34) + +- CometProject (33) + +- CometSortMergeJoin (32) + :- CometSort (17) + : +- CometExchange (16) + : +- CometProject (15) + : +- CometWindowExec (14) + : +- CometSort (13) + : +- CometExchange (12) + : +- CometHashAggregate (11) + : +- CometExchange (10) + : +- CometHashAggregate (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- CometSort (31) + +- CometExchange (30) + +- CometProject (29) + +- CometWindowExec (28) + +- CometSort (27) + +- CometExchange (26) + +- CometHashAggregate (25) + +- CometExchange (24) + +- CometHashAggregate (23) + +- CometProject (22) + +- CometBroadcastHashJoin (21) + :- CometFilter (19) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) + +- ReusedExchange (20) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -104,26 +102,23 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) CometColumnarToRow [codegen id : 1] +(14) CometWindowExec Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11], [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Window -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 2] -Output [3]: [item_sk#9, d_date#6, cume_sales#11] +(15) CometProject Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] +Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9, d_date#6, cume_sales#11] -(17) CometColumnarExchange +(16) CometExchange Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(18) CometSort +(17) CometSort Input [3]: [item_sk#9, d_date#6, cume_sales#11] Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] @@ -131,127 +126,124 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(20) CometFilter +(19) CometFilter Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Condition : isnotnull(ss_item_sk#12) -(21) ReusedExchange [Reuses operator id: 6] +(20) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#16, d_date#17] -(22) CometBroadcastHashJoin +(21) CometBroadcastHashJoin Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Right output [2]: [d_date_sk#16, d_date#17] Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight -(23) CometProject +(22) CometProject Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] -(24) CometHashAggregate +(23) CometHashAggregate Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] Keys [2]: [ss_item_sk#12, d_date#17] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] -(25) CometExchange +(24) CometExchange Input [3]: [ss_item_sk#12, d_date#17, sum#18] Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(26) CometHashAggregate +(25) CometHashAggregate Input [3]: [ss_item_sk#12, d_date#17, sum#18] Keys [2]: [ss_item_sk#12, d_date#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] -(27) CometExchange +(26) CometExchange Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(28) CometSort +(27) CometSort Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(29) CometColumnarToRow [codegen id : 3] -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] - -(30) Window +(28) CometWindowExec Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] +Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21], [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] -(31) Project [codegen id : 4] -Output [3]: [item_sk#19, d_date#17, cume_sales#21] +(29) CometProject Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] +Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19, d_date#17, cume_sales#21] -(32) CometColumnarExchange +(30) CometExchange Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(33) CometSort +(31) CometSort Input [3]: [item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(34) CometSortMergeJoin +(32) CometSortMergeJoin Left output [3]: [item_sk#9, d_date#6, cume_sales#11] Right output [3]: [item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter -(35) CometProject +(33) CometProject Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] -(36) CometExchange +(34) CometExchange Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(37) CometSort +(35) CometSort Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] -(38) CometColumnarToRow [codegen id : 5] +(36) CometWindowExec Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] -(39) Window -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] - -(40) Filter [codegen id : 6] +(37) CometFilter Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) -(41) TakeOrderedAndProject +(38) CometTakeOrderedAndProject +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#22 ASC NULLS FIRST,d_date#23 ASC NULLS FIRST], output=[item_sk#22,d_date#23,web_sales#24,store_sales#25,web_cumulative#26,store_cumulative#27]), [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], 100, 0, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] + +(39) CometColumnarToRow [codegen id : 1] Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) +BroadcastExchange (44) ++- * CometColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter +(41) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(44) CometProject +(42) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(45) CometColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(46) BroadcastExchange +(44) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt index 8d4877cb50..3c719aa579 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt @@ -1,53 +1,51 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 47 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt index a5af2e114b..9215d1d12f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt @@ -1,62 +1,50 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (6) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (2) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #6 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometWindowExec [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #2 + CometProject [item_sk,d_date,cume_sales] + CometWindowExec [item_sk,d_date,_w0,ws_item_sk,cume_sales] + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometExchange [ws_item_sk] #3 + CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #4 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #6 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #7 + CometProject [item_sk,d_date,cume_sales] + CometWindowExec [item_sk,d_date,_w0,ss_item_sk,cume_sales] + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometExchange [ss_item_sk] #8 + CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #9 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt index 683cc25d24..4c7d196606 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt @@ -1,45 +1,43 @@ == Physical Plan == -TakeOrderedAndProject (41) -+- * Filter (40) - +- Window (39) - +- * CometColumnarToRow (38) - +- CometSort (37) - +- CometExchange (36) - +- CometProject (35) - +- CometSortMergeJoin (34) - :- CometSort (18) - : +- CometColumnarExchange (17) - : +- * Project (16) - : +- Window (15) - : +- * CometColumnarToRow (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (33) - +- CometColumnarExchange (32) - +- * Project (31) - +- Window (30) - +- * CometColumnarToRow (29) - +- CometSort (28) - +- CometExchange (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) +* CometColumnarToRow (39) ++- CometTakeOrderedAndProject (38) + +- CometFilter (37) + +- CometWindowExec (36) + +- CometSort (35) + +- CometExchange (34) + +- CometProject (33) + +- CometSortMergeJoin (32) + :- CometSort (17) + : +- CometExchange (16) + : +- CometProject (15) + : +- CometWindowExec (14) + : +- CometSort (13) + : +- CometExchange (12) + : +- CometHashAggregate (11) + : +- CometExchange (10) + : +- CometHashAggregate (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- CometSort (31) + +- CometExchange (30) + +- CometProject (29) + +- CometWindowExec (28) + +- CometSort (27) + +- CometExchange (26) + +- CometHashAggregate (25) + +- CometExchange (24) + +- CometHashAggregate (23) + +- CometProject (22) + +- CometBroadcastHashJoin (21) + :- CometFilter (19) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) + +- ReusedExchange (20) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -104,26 +102,23 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) CometColumnarToRow [codegen id : 1] +(14) CometWindowExec Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11], [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Window -Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 2] -Output [3]: [item_sk#9, d_date#6, cume_sales#11] +(15) CometProject Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] +Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9, d_date#6, cume_sales#11] -(17) CometColumnarExchange +(16) CometExchange Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(18) CometSort +(17) CometSort Input [3]: [item_sk#9, d_date#6, cume_sales#11] Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] @@ -131,127 +126,124 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(20) CometFilter +(19) CometFilter Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Condition : isnotnull(ss_item_sk#12) -(21) ReusedExchange [Reuses operator id: 6] +(20) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#16, d_date#17] -(22) CometBroadcastHashJoin +(21) CometBroadcastHashJoin Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Right output [2]: [d_date_sk#16, d_date#17] Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight -(23) CometProject +(22) CometProject Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] -(24) CometHashAggregate +(23) CometHashAggregate Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] Keys [2]: [ss_item_sk#12, d_date#17] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] -(25) CometExchange +(24) CometExchange Input [3]: [ss_item_sk#12, d_date#17, sum#18] Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(26) CometHashAggregate +(25) CometHashAggregate Input [3]: [ss_item_sk#12, d_date#17, sum#18] Keys [2]: [ss_item_sk#12, d_date#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] -(27) CometExchange +(26) CometExchange Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(28) CometSort +(27) CometSort Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(29) CometColumnarToRow [codegen id : 3] -Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] - -(30) Window +(28) CometWindowExec Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] +Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21], [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] -(31) Project [codegen id : 4] -Output [3]: [item_sk#19, d_date#17, cume_sales#21] +(29) CometProject Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] +Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19, d_date#17, cume_sales#21] -(32) CometColumnarExchange +(30) CometExchange Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(33) CometSort +(31) CometSort Input [3]: [item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(34) CometSortMergeJoin +(32) CometSortMergeJoin Left output [3]: [item_sk#9, d_date#6, cume_sales#11] Right output [3]: [item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter -(35) CometProject +(33) CometProject Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] -(36) CometExchange +(34) CometExchange Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(37) CometSort +(35) CometSort Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] -(38) CometColumnarToRow [codegen id : 5] +(36) CometWindowExec Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] -(39) Window -Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] - -(40) Filter [codegen id : 6] +(37) CometFilter Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) -(41) TakeOrderedAndProject +(38) CometTakeOrderedAndProject +Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#22 ASC NULLS FIRST,d_date#23 ASC NULLS FIRST], output=[item_sk#22,d_date#23,web_sales#24,store_sales#25,web_cumulative#26,store_cumulative#27]), [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], 100, 0, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] + +(39) CometColumnarToRow [codegen id : 1] Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) +BroadcastExchange (44) ++- * CometColumnarToRow (43) + +- CometProject (42) + +- CometFilter (41) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter +(41) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(44) CometProject +(42) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(45) CometColumnarToRow [codegen id : 1] +(43) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(46) BroadcastExchange +(44) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt index 8d4877cb50..3c719aa579 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt @@ -1,53 +1,51 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 44 out of 47 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index a5af2e114b..9215d1d12f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -1,62 +1,50 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (6) - Filter [web_cumulative,store_cumulative] - InputAdapter - Window [web_sales,item_sk,d_date,store_sales] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (2) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #7 - WholeStageCodegen (4) - Project [item_sk,d_date,cume_sales] - InputAdapter - Window [_w0,ss_item_sk,d_date] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #6 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometWindowExec [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #2 + CometProject [item_sk,d_date,cume_sales] + CometWindowExec [item_sk,d_date,_w0,ws_item_sk,cume_sales] + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometExchange [ws_item_sk] #3 + CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #4 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #6 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #7 + CometProject [item_sk,d_date,cume_sales] + CometWindowExec [item_sk,d_date,_w0,ss_item_sk,cume_sales] + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometExchange [ss_item_sk] #8 + CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #9 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt index 2fc38bf838..0dfd5e81c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) +* CometColumnarToRow (29) ++- CometTakeOrderedAndProject (28) + +- CometProject (27) + +- CometFilter (26) + +- CometWindowExec (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativ Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 1] +(25) CometWindowExec Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22], [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] -(26) Window -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] - -(27) Filter [codegen id : 2] +(26) CometFilter Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END -(28) Project [codegen id : 2] -Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +(27) CometProject Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] +Arguments: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] + +(28) CometTakeOrderedAndProject +Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[avg_quarterly_sales#22 ASC NULLS FIRST,sum_sales#20 ASC NULLS FIRST,i_manufact_id#5 ASC NULLS FIRST], output=[i_manufact_id#5,sum_sales#20,avg_quarterly_sales#22]), [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22], 100, 0, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -(29) TakeOrderedAndProject +(29) CometColumnarToRow [codegen id : 1] Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt index 07ad98fa22..8adffb63e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt index 0c123eb728..0a2d088fd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt @@ -1,41 +1,39 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (2) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometExchange [i_manufact_id] #1 - CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manufact_id,sum_sales,avg_quarterly_sales] + CometProject [i_manufact_id,sum_sales,avg_quarterly_sales] + CometFilter [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] + CometWindowExec [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] + CometSort [i_manufact_id,sum_sales,_w0] + CometExchange [i_manufact_id] #1 + CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 2fc38bf838..0dfd5e81c4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) +* CometColumnarToRow (29) ++- CometTakeOrderedAndProject (28) + +- CometProject (27) + +- CometFilter (26) + +- CometWindowExec (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativ Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 1] +(25) CometWindowExec Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22], [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] -(26) Window -Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] - -(27) Filter [codegen id : 2] +(26) CometFilter Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END -(28) Project [codegen id : 2] -Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +(27) CometProject Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] +Arguments: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] + +(28) CometTakeOrderedAndProject +Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[avg_quarterly_sales#22 ASC NULLS FIRST,sum_sales#20 ASC NULLS FIRST,i_manufact_id#5 ASC NULLS FIRST], output=[i_manufact_id#5,sum_sales#20,avg_quarterly_sales#22]), [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22], 100, 0, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -(29) TakeOrderedAndProject +(29) CometColumnarToRow [codegen id : 1] Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt index 07ad98fa22..8adffb63e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index 0c123eb728..0a2d088fd8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -1,41 +1,39 @@ -TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] - WholeStageCodegen (2) - Project [i_manufact_id,sum_sales,avg_quarterly_sales] - Filter [avg_quarterly_sales,sum_sales] - InputAdapter - Window [_w0,i_manufact_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manufact_id,sum_sales,_w0] - CometExchange [i_manufact_id] #1 - CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manufact_id,sum_sales,avg_quarterly_sales] + CometProject [i_manufact_id,sum_sales,avg_quarterly_sales] + CometFilter [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] + CometWindowExec [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] + CometSort [i_manufact_id,sum_sales,_w0] + CometExchange [i_manufact_id] #1 + CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt index eb21a3abd7..9d0b44c8c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt @@ -1,52 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (28) + : : +- CometFilter (27) + : : +- CometWindowExec (26) + : : +- CometFilter (25) + : : +- CometWindowExec (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometWindowExec (33) + : +- CometSort (32) + : +- CometExchange (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + +- ReusedExchange (38) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -159,131 +153,107 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] +(24) CometWindowExec Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] +(25) CometFilter Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window +(26) CometWindowExec Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20], [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(28) Filter [codegen id : 7] +(27) CometFilter Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] +(28) CometProject Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -(30) ReusedExchange [Reuses operator id: 20] +(29) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -(31) CometHashAggregate +(30) CometHashAggregate Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -(32) CometExchange +(31) CometExchange Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(33) CometSort +(32) CometSort Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window +(33) CometWindowExec Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26], [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] +(34) CometProject Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] +Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26], [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -(37) BroadcastExchange +(35) CometBroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] +Right output [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)], Inner, BuildRight -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +(37) CometProject Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] +(38) ReusedExchange [Reuses operator id: 35] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +Right output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#30, i_brand#31, cc_name#32, (rn#34 - 1)], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None +(40) CometProject +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#35, sum_sales#33 AS nsum#36] -(47) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +(41) CometTakeOrderedAndProject +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST,cc_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,cc_name#15,d_year#12,d_moy#13,avg_monthly_sales#20,sum_sales#17,psum#35,nsum#36]), [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], 100, 0, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] -(48) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +(42) CometColumnarToRow [codegen id : 1] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(44) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(46) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt index 101b15d3ac..00de5f32a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt @@ -1,78 +1,76 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt index b854e818db..c0f7f57f6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt @@ -1,71 +1,51 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,cc_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,cc_name] #8 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index eb21a3abd7..9d0b44c8c2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -1,52 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (28) + : : +- CometFilter (27) + : : +- CometWindowExec (26) + : : +- CometFilter (25) + : : +- CometWindowExec (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometWindowExec (33) + : +- CometSort (32) + : +- CometExchange (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + +- ReusedExchange (38) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -159,131 +153,107 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] +(24) CometWindowExec Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] +(25) CometFilter Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window +(26) CometWindowExec Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20], [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(28) Filter [codegen id : 7] +(27) CometFilter Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] +(28) CometProject Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -(30) ReusedExchange [Reuses operator id: 20] +(29) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -(31) CometHashAggregate +(30) CometHashAggregate Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -(32) CometExchange +(31) CometExchange Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(33) CometSort +(32) CometSort Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window +(33) CometWindowExec Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26], [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] +(34) CometProject Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] +Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26], [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -(37) BroadcastExchange +(35) CometBroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] +Right output [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)], Inner, BuildRight -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +(37) CometProject Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] +(38) ReusedExchange [Reuses operator id: 35] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +Right output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#30, i_brand#31, cc_name#32, (rn#34 - 1)], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None +(40) CometProject +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#35, sum_sales#33 AS nsum#36] -(47) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +(41) CometTakeOrderedAndProject +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST,cc_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,cc_name#15,d_year#12,d_moy#13,avg_monthly_sales#20,sum_sales#17,psum#35,nsum#36]), [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], 100, 0, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] -(48) TakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +(42) CometColumnarToRow [codegen id : 1] +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(44) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(46) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt index 101b15d3ac..00de5f32a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt @@ -1,78 +1,76 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index b854e818db..c0f7f57f6d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -1,71 +1,51 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,cc_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,cc_name] #8 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt index e2a0cceb1a..aa9e4d43d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) +* CometColumnarToRow (29) ++- CometTakeOrderedAndProject (28) + +- CometProject (27) + +- CometFilter (26) + +- CometWindowExec (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNative Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 1] +(25) CometWindowExec Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22], [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] -(26) Window -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] - -(27) Filter [codegen id : 2] +(26) CometFilter Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END -(28) Project [codegen id : 2] -Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +(27) CometProject Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] +Arguments: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] + +(28) CometTakeOrderedAndProject +Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_manager_id#5 ASC NULLS FIRST,avg_monthly_sales#22 ASC NULLS FIRST,sum_sales#20 ASC NULLS FIRST], output=[i_manager_id#5,sum_sales#20,avg_monthly_sales#22]), [i_manager_id#5, sum_sales#20, avg_monthly_sales#22], 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -(29) TakeOrderedAndProject +(29) CometColumnarToRow [codegen id : 1] Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt index 07ad98fa22..8adffb63e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt index 02166879a9..acdc4ee744 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt @@ -1,41 +1,39 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (2) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometExchange [i_manager_id] #1 - CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manager_id,sum_sales,avg_monthly_sales] + CometProject [i_manager_id,sum_sales,avg_monthly_sales] + CometFilter [i_manager_id,sum_sales,_w0,avg_monthly_sales] + CometWindowExec [i_manager_id,sum_sales,_w0,avg_monthly_sales] + CometSort [i_manager_id,sum_sales,_w0] + CometExchange [i_manager_id] #1 + CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manager_id,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index e2a0cceb1a..aa9e4d43d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) +* CometColumnarToRow (29) ++- CometTakeOrderedAndProject (28) + +- CometProject (27) + +- CometFilter (26) + +- CometWindowExec (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNative Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 1] +(25) CometWindowExec Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22], [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] -(26) Window -Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] - -(27) Filter [codegen id : 2] +(26) CometFilter Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END -(28) Project [codegen id : 2] -Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +(27) CometProject Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] +Arguments: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] + +(28) CometTakeOrderedAndProject +Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_manager_id#5 ASC NULLS FIRST,avg_monthly_sales#22 ASC NULLS FIRST,sum_sales#20 ASC NULLS FIRST], output=[i_manager_id#5,sum_sales#20,avg_monthly_sales#22]), [i_manager_id#5, sum_sales#20, avg_monthly_sales#22], 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -(29) TakeOrderedAndProject +(29) CometColumnarToRow [codegen id : 1] Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt index 07ad98fa22..8adffb63e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index 02166879a9..acdc4ee744 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -1,41 +1,39 @@ -TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] - WholeStageCodegen (2) - Project [i_manager_id,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_manager_id] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_manager_id,sum_sales,_w0] - CometExchange [i_manager_id] #1 - CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_manager_id,sum_sales,avg_monthly_sales] + CometProject [i_manager_id,sum_sales,avg_monthly_sales] + CometFilter [i_manager_id,sum_sales,_w0,avg_monthly_sales] + CometWindowExec [i_manager_id,sum_sales,_w0,avg_monthly_sales] + CometSort [i_manager_id,sum_sales,_w0] + CometExchange [i_manager_id] #1 + CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manager_id,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt index df94b5d986..151d5e1afa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Filter (29) - +- Window (28) - +- * CometColumnarToRow (27) +* CometColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometFilter (28) + +- CometWindowExec (27) +- CometSort (26) +- CometExchange (25) +- CometHashAggregate (24) @@ -153,20 +153,20 @@ Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometNativeS Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] -(27) CometColumnarToRow [codegen id : 1] +(27) CometWindowExec Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] +Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36], [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] -(28) Window -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] - -(29) Filter [codegen id : 2] +(28) CometFilter Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] Condition : (rk#36 <= 100) -(30) TakeOrderedAndProject +(29) CometTakeOrderedAndProject +Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#24 ASC NULLS FIRST,i_class#25 ASC NULLS FIRST,i_brand#26 ASC NULLS FIRST,i_product_name#27 ASC NULLS FIRST,d_year#28 ASC NULLS FIRST,d_qoy#29 ASC NULLS FIRST,d_moy#30 ASC NULLS FIRST,s_store_id#31 ASC NULLS FIRST,sumsales#35 ASC NULLS FIRST,rk#36 ASC NULLS FIRST], output=[i_category#24,i_class#25,i_brand#26,i_product_name#27,d_year#28,d_qoy#29,d_moy#30,s_store_id#31,sumsales#35,rk#36]), [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36], 100, 0, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] + +(30) CometColumnarToRow [codegen id : 1] Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt index 038a03a451..da844f8a5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -35,4 +35,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 34 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt index e85c243077..f6ddea4c08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt @@ -1,42 +1,40 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (2) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] - CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] - CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometFilter [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometWindowExec [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometExchange [i_category] #1 + CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] + CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] + CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index df94b5d986..151d5e1afa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (30) -+- * Filter (29) - +- Window (28) - +- * CometColumnarToRow (27) +* CometColumnarToRow (30) ++- CometTakeOrderedAndProject (29) + +- CometFilter (28) + +- CometWindowExec (27) +- CometSort (26) +- CometExchange (25) +- CometHashAggregate (24) @@ -153,20 +153,20 @@ Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometNativeS Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] -(27) CometColumnarToRow [codegen id : 1] +(27) CometWindowExec Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] +Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36], [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] -(28) Window -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] - -(29) Filter [codegen id : 2] +(28) CometFilter Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] Condition : (rk#36 <= 100) -(30) TakeOrderedAndProject +(29) CometTakeOrderedAndProject +Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#24 ASC NULLS FIRST,i_class#25 ASC NULLS FIRST,i_brand#26 ASC NULLS FIRST,i_product_name#27 ASC NULLS FIRST,d_year#28 ASC NULLS FIRST,d_qoy#29 ASC NULLS FIRST,d_moy#30 ASC NULLS FIRST,s_store_id#31 ASC NULLS FIRST,sumsales#35 ASC NULLS FIRST,rk#36 ASC NULLS FIRST], output=[i_category#24,i_class#25,i_brand#26,i_product_name#27,d_year#28,d_qoy#29,d_moy#30,s_store_id#31,sumsales#35,rk#36]), [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36], 100, 0, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] + +(30) CometColumnarToRow [codegen id : 1] Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt index 038a03a451..da844f8a5a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -35,4 +35,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 34 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index e85c243077..f6ddea4c08 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -1,42 +1,40 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (2) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] - CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] - CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometFilter [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometWindowExec [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometExchange [i_category] #1 + CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] + CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] + CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt index d3610af0f8..d059dabb26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt @@ -1,52 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Expand (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * CometColumnarToRow (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- BroadcastExchange (35) - +- * Project (34) - +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * CometColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- Window (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- ReusedExchange (21) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometProject (42) + +- CometWindowExec (41) + +- CometSort (40) + +- CometExchange (39) + +- CometHashAggregate (38) + +- CometExchange (37) + +- CometHashAggregate (36) + +- CometExpand (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) + +- CometBroadcastExchange (29) + +- CometProject (28) + +- CometFilter (27) + +- CometWindowExec (26) + +- CometSort (25) + +- CometHashAggregate (24) + +- CometExchange (23) + +- CometHashAggregate (22) + +- CometProject (21) + +- CometBroadcastHashJoin (20) + :- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometFilter (12) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) + +- ReusedExchange (19) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -89,24 +85,18 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] @@ -114,191 +104,179 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Condition : isnotnull(ss_store_sk#10) -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [2]: [s_store_sk#7, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [2]: [s_store_sk#7, s_state#9] Condition : isnotnull(s_store_sk#7) -(17) CometProject +(15) CometProject Input [2]: [s_store_sk#7, s_state#9] Arguments: [s_store_sk#7, s_state#14], [s_store_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#14] -(18) CometBroadcastExchange +(16) CometBroadcastExchange Input [2]: [s_store_sk#7, s_state#14] Arguments: [s_store_sk#7, s_state#14] -(19) CometBroadcastHashJoin +(17) CometBroadcastHashJoin Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Right output [2]: [s_store_sk#7, s_state#14] Arguments: [ss_store_sk#10], [s_store_sk#7], Inner, BuildRight -(20) CometProject +(18) CometProject Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#7, s_state#14] Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14], [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] -(21) ReusedExchange [Reuses operator id: 6] +(19) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#5] -(22) CometBroadcastHashJoin +(20) CometBroadcastHashJoin Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] Right output [1]: [d_date_sk#5] Arguments: [ss_sold_date_sk#12], [d_date_sk#5], Inner, BuildRight -(23) CometProject +(21) CometProject Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14, d_date_sk#5] Arguments: [ss_net_profit#11, s_state#14], [ss_net_profit#11, s_state#14] -(24) CometHashAggregate +(22) CometHashAggregate Input [2]: [ss_net_profit#11, s_state#14] Keys [1]: [s_state#14] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] -(25) CometExchange +(23) CometExchange Input [2]: [s_state#14, sum#15] Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(26) CometHashAggregate +(24) CometHashAggregate Input [2]: [s_state#14, sum#15] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -(27) CometSort +(25) CometSort Input [3]: [s_state#14, _w0#16, s_state#14] Arguments: [s_state#14, _w0#16, s_state#14], [s_state#14 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] -(28) CometColumnarToRow [codegen id : 1] +(26) CometWindowExec Input [3]: [s_state#14, _w0#16, s_state#14] +Arguments: [s_state#14, _w0#16, s_state#14, ranking#17], [rank(_w0#16) windowspecdefinition(s_state#14, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#14], [_w0#16 DESC NULLS LAST] -(29) Window -Input [3]: [s_state#14, _w0#16, s_state#14] -Arguments: [rank(_w0#16) windowspecdefinition(s_state#14, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#14], [_w0#16 DESC NULLS LAST] - -(30) Filter [codegen id : 2] +(27) CometFilter Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] Condition : (ranking#17 <= 5) -(31) Project [codegen id : 2] -Output [1]: [s_state#14] +(28) CometProject Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] +Arguments: [s_state#14], [s_state#14] -(32) BroadcastExchange +(29) CometBroadcastExchange Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] +Arguments: [s_state#14] -(33) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#14] -Join type: LeftSemi -Join condition: None +(30) CometBroadcastHashJoin +Left output [3]: [s_store_sk#7, s_county#8, s_state#9] +Right output [1]: [s_state#14] +Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)], [s_state#14], LeftSemi, BuildRight -(34) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#18] +(31) CometProject Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Arguments: [s_store_sk#7, s_county#8, s_state#18], [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#18] -(35) BroadcastExchange +(32) CometBroadcastExchange Input [3]: [s_store_sk#7, s_county#8, s_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [s_store_sk#7, s_county#8, s_state#18] -(36) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None +(33) CometBroadcastHashJoin +Left output [2]: [ss_store_sk#1, ss_net_profit#2] +Right output [3]: [s_store_sk#7, s_county#8, s_state#18] +Arguments: [ss_store_sk#1], [s_store_sk#7], Inner, BuildRight -(37) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_state#18, s_county#8] +(34) CometProject Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#18] +Arguments: [ss_net_profit#2, s_state#18, s_county#8], [ss_net_profit#2, s_state#18, s_county#8] -(38) Expand [codegen id : 4] +(35) CometExpand Input [3]: [ss_net_profit#2, s_state#18, s_county#8] Arguments: [[ss_net_profit#2, s_state#18, s_county#8, 0], [ss_net_profit#2, s_state#18, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] -(39) HashAggregate [codegen id : 4] +(36) CometHashAggregate Input [4]: [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] - -(40) CometColumnarExchange -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] -Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 5] -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] +(37) CometExchange +Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#22] +Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) HashAggregate [codegen id : 5] -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] +(38) CometHashAggregate +Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#22] Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS total_sum#25, s_state#19, s_county#20, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS lochierarchy#26, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS _w0#27, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS _w1#28, CASE WHEN (cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) = 0) THEN s_state#19 END AS _w2#29] -(43) CometColumnarExchange -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: hashpartitioning(_w1#28, _w2#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(39) CometExchange +Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: hashpartitioning(_w1#26, _w2#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(44) CometSort -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29], [_w1#28 ASC NULLS FIRST, _w2#29 ASC NULLS FIRST, _w0#27 DESC NULLS LAST] +(40) CometSort +Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27], [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 DESC NULLS LAST] -(45) CometColumnarToRow [codegen id : 6] -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] +(41) CometWindowExec +Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28], [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 DESC NULLS LAST] -(46) Window -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [rank(_w0#27) windowspecdefinition(_w1#28, _w2#29, _w0#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#30], [_w1#28, _w2#29], [_w0#27 DESC NULLS LAST] +(42) CometProject +Input [8]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] +Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28], [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] -(47) Project [codegen id : 7] -Output [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] -Input [8]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29, rank_within_parent#30] +(43) CometTakeOrderedAndProject +Input [5]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#24 DESC NULLS LAST,CASE WHEN (lochierarchy#24 = 0) THEN s_state#19 END ASC NULLS FIRST,rank_within_parent#28 ASC NULLS FIRST], output=[total_sum#23,s_state#19,s_county#20,lochierarchy#24,rank_within_parent#28]), [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28], 100, 0, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN s_state#19 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] -(48) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] -Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#19 END ASC NULLS FIRST, rank_within_parent#30 ASC NULLS FIRST], [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] +(44) CometColumnarToRow [codegen id : 1] +Input [5]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (49) ++- * CometColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(46) CometFilter Input [2]: [d_date_sk#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(51) CometProject +(47) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(52) CometColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(53) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt index 3e666f40ed..e0eb970463 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt @@ -1,60 +1,56 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometHashAggregate + +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt index d6ba61a844..b16765535b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt @@ -1,73 +1,55 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (7) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #4 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum] [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),spark_grouping_id] + CometExchange [s_state,s_county,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit] [s_state,s_county,spark_grouping_id,sum] + CometExpand [s_state,s_county] [ss_net_profit,s_state,s_county,spark_grouping_id] + CometProject [ss_net_profit,s_state,s_county] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_county,s_state] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [s_store_sk,s_county,s_state] #5 + CometProject [s_state] [s_store_sk,s_county,s_state] + CometBroadcastHashJoin [s_store_sk,s_county,s_state,s_state] + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + CometBroadcastExchange [s_state] #6 + CometProject [s_state] + CometFilter [s_state,_w0,ranking] + CometWindowExec [s_state,_w0,ranking] + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] + CometExchange [s_state] #7 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index d3610af0f8..d059dabb26 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -1,52 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) - +- CometSort (44) - +- CometColumnarExchange (43) - +- * HashAggregate (42) - +- * CometColumnarToRow (41) - +- CometColumnarExchange (40) - +- * HashAggregate (39) - +- * Expand (38) - +- * Project (37) - +- * BroadcastHashJoin Inner BuildRight (36) - :- * CometColumnarToRow (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- BroadcastExchange (35) - +- * Project (34) - +- * BroadcastHashJoin LeftSemi BuildRight (33) - :- * CometColumnarToRow (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - +- BroadcastExchange (32) - +- * Project (31) - +- * Filter (30) - +- Window (29) - +- * CometColumnarToRow (28) - +- CometSort (27) - +- CometHashAggregate (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometProject (20) - : +- CometBroadcastHashJoin (19) - : :- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : +- CometBroadcastExchange (18) - : +- CometProject (17) - : +- CometFilter (16) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - +- ReusedExchange (21) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometProject (42) + +- CometWindowExec (41) + +- CometSort (40) + +- CometExchange (39) + +- CometHashAggregate (38) + +- CometExchange (37) + +- CometHashAggregate (36) + +- CometExpand (35) + +- CometProject (34) + +- CometBroadcastHashJoin (33) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- CometBroadcastExchange (32) + +- CometProject (31) + +- CometBroadcastHashJoin (30) + :- CometFilter (10) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) + +- CometBroadcastExchange (29) + +- CometProject (28) + +- CometFilter (27) + +- CometWindowExec (26) + +- CometSort (25) + +- CometHashAggregate (24) + +- CometExchange (23) + +- CometHashAggregate (22) + +- CometProject (21) + +- CometBroadcastHashJoin (20) + :- CometProject (18) + : +- CometBroadcastHashJoin (17) + : :- CometFilter (12) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) + : +- CometBroadcastExchange (16) + : +- CometProject (15) + : +- CometFilter (14) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) + +- ReusedExchange (19) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -89,24 +85,18 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] @@ -114,191 +104,179 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Condition : isnotnull(ss_store_sk#10) -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [2]: [s_store_sk#7, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [2]: [s_store_sk#7, s_state#9] Condition : isnotnull(s_store_sk#7) -(17) CometProject +(15) CometProject Input [2]: [s_store_sk#7, s_state#9] Arguments: [s_store_sk#7, s_state#14], [s_store_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#14] -(18) CometBroadcastExchange +(16) CometBroadcastExchange Input [2]: [s_store_sk#7, s_state#14] Arguments: [s_store_sk#7, s_state#14] -(19) CometBroadcastHashJoin +(17) CometBroadcastHashJoin Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Right output [2]: [s_store_sk#7, s_state#14] Arguments: [ss_store_sk#10], [s_store_sk#7], Inner, BuildRight -(20) CometProject +(18) CometProject Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#7, s_state#14] Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14], [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] -(21) ReusedExchange [Reuses operator id: 6] +(19) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#5] -(22) CometBroadcastHashJoin +(20) CometBroadcastHashJoin Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] Right output [1]: [d_date_sk#5] Arguments: [ss_sold_date_sk#12], [d_date_sk#5], Inner, BuildRight -(23) CometProject +(21) CometProject Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14, d_date_sk#5] Arguments: [ss_net_profit#11, s_state#14], [ss_net_profit#11, s_state#14] -(24) CometHashAggregate +(22) CometHashAggregate Input [2]: [ss_net_profit#11, s_state#14] Keys [1]: [s_state#14] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] -(25) CometExchange +(23) CometExchange Input [2]: [s_state#14, sum#15] Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(26) CometHashAggregate +(24) CometHashAggregate Input [2]: [s_state#14, sum#15] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -(27) CometSort +(25) CometSort Input [3]: [s_state#14, _w0#16, s_state#14] Arguments: [s_state#14, _w0#16, s_state#14], [s_state#14 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] -(28) CometColumnarToRow [codegen id : 1] +(26) CometWindowExec Input [3]: [s_state#14, _w0#16, s_state#14] +Arguments: [s_state#14, _w0#16, s_state#14, ranking#17], [rank(_w0#16) windowspecdefinition(s_state#14, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#14], [_w0#16 DESC NULLS LAST] -(29) Window -Input [3]: [s_state#14, _w0#16, s_state#14] -Arguments: [rank(_w0#16) windowspecdefinition(s_state#14, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#14], [_w0#16 DESC NULLS LAST] - -(30) Filter [codegen id : 2] +(27) CometFilter Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] Condition : (ranking#17 <= 5) -(31) Project [codegen id : 2] -Output [1]: [s_state#14] +(28) CometProject Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] +Arguments: [s_state#14], [s_state#14] -(32) BroadcastExchange +(29) CometBroadcastExchange Input [1]: [s_state#14] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] +Arguments: [s_state#14] -(33) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#14] -Join type: LeftSemi -Join condition: None +(30) CometBroadcastHashJoin +Left output [3]: [s_store_sk#7, s_county#8, s_state#9] +Right output [1]: [s_state#14] +Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)], [s_state#14], LeftSemi, BuildRight -(34) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#18] +(31) CometProject Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Arguments: [s_store_sk#7, s_county#8, s_state#18], [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#18] -(35) BroadcastExchange +(32) CometBroadcastExchange Input [3]: [s_store_sk#7, s_county#8, s_state#18] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [s_store_sk#7, s_county#8, s_state#18] -(36) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None +(33) CometBroadcastHashJoin +Left output [2]: [ss_store_sk#1, ss_net_profit#2] +Right output [3]: [s_store_sk#7, s_county#8, s_state#18] +Arguments: [ss_store_sk#1], [s_store_sk#7], Inner, BuildRight -(37) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_state#18, s_county#8] +(34) CometProject Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#18] +Arguments: [ss_net_profit#2, s_state#18, s_county#8], [ss_net_profit#2, s_state#18, s_county#8] -(38) Expand [codegen id : 4] +(35) CometExpand Input [3]: [ss_net_profit#2, s_state#18, s_county#8] Arguments: [[ss_net_profit#2, s_state#18, s_county#8, 0], [ss_net_profit#2, s_state#18, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] -(39) HashAggregate [codegen id : 4] +(36) CometHashAggregate Input [4]: [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] - -(40) CometColumnarExchange -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] -Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(41) CometColumnarToRow [codegen id : 5] -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] +(37) CometExchange +Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#22] +Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) HashAggregate [codegen id : 5] -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] +(38) CometHashAggregate +Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#22] Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS total_sum#25, s_state#19, s_county#20, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS lochierarchy#26, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS _w0#27, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS _w1#28, CASE WHEN (cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) = 0) THEN s_state#19 END AS _w2#29] -(43) CometColumnarExchange -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: hashpartitioning(_w1#28, _w2#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(39) CometExchange +Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: hashpartitioning(_w1#26, _w2#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(44) CometSort -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29], [_w1#28 ASC NULLS FIRST, _w2#29 ASC NULLS FIRST, _w0#27 DESC NULLS LAST] +(40) CometSort +Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27], [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 DESC NULLS LAST] -(45) CometColumnarToRow [codegen id : 6] -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] +(41) CometWindowExec +Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28], [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 DESC NULLS LAST] -(46) Window -Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -Arguments: [rank(_w0#27) windowspecdefinition(_w1#28, _w2#29, _w0#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#30], [_w1#28, _w2#29], [_w0#27 DESC NULLS LAST] +(42) CometProject +Input [8]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] +Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28], [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] -(47) Project [codegen id : 7] -Output [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] -Input [8]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29, rank_within_parent#30] +(43) CometTakeOrderedAndProject +Input [5]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#24 DESC NULLS LAST,CASE WHEN (lochierarchy#24 = 0) THEN s_state#19 END ASC NULLS FIRST,rank_within_parent#28 ASC NULLS FIRST], output=[total_sum#23,s_state#19,s_county#20,lochierarchy#24,rank_within_parent#28]), [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28], 100, 0, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN s_state#19 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] -(48) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] -Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#19 END ASC NULLS FIRST, rank_within_parent#30 ASC NULLS FIRST], [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] +(44) CometColumnarToRow [codegen id : 1] +Input [5]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (53) -+- * CometColumnarToRow (52) - +- CometProject (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (49) ++- * CometColumnarToRow (48) + +- CometProject (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(46) CometFilter Input [2]: [d_date_sk#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(51) CometProject +(47) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(52) CometColumnarToRow [codegen id : 1] +(48) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(53) BroadcastExchange +(49) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt index 3e666f40ed..e0eb970463 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt @@ -1,60 +1,56 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Expand - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometExpand + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometHashAggregate + +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index d6ba61a844..b16765535b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -1,73 +1,55 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (7) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #4 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum] [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),spark_grouping_id] + CometExchange [s_state,s_county,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit] [s_state,s_county,spark_grouping_id,sum] + CometExpand [s_state,s_county] [ss_net_profit,s_state,s_county,spark_grouping_id] + CometProject [ss_net_profit,s_state,s_county] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_county,s_state] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [s_store_sk,s_county,s_state] #5 + CometProject [s_state] [s_store_sk,s_county,s_state] + CometBroadcastHashJoin [s_store_sk,s_county,s_state,s_state] + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + CometBroadcastExchange [s_state] #6 + CometProject [s_state] + CometFilter [s_state,_w0,ranking] + CometWindowExec [s_state,_w0,ranking] + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] + CometExchange [s_state] #7 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt index 1af5e2612c..20b60287be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometProject (22) + +- CometWindowExec (21) +- CometSort (20) +- CometExchange (19) +- CometHashAggregate (18) @@ -119,20 +119,20 @@ Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNative Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(21) CometColumnarToRow [codegen id : 1] +(21) CometWindowExec Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21], [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] -(22) Window -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] - -(23) Project [codegen id : 2] -Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +(22) CometProject Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] +Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] + +(23) CometTakeOrderedAndProject +Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#17 DESC NULLS LAST,CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST,rank_within_parent#21 ASC NULLS FIRST], output=[total_sum#16,i_category#12,i_class#13,lochierarchy#17,rank_within_parent#21]), [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21], 100, 0, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -(24) TakeOrderedAndProject +(24) CometColumnarToRow [codegen id : 1] Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt index d5ebd5a7e7..f919130b53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -29,4 +29,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt index 29e2d72920..a6b54bd89f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt @@ -1,36 +1,34 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] - CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt index 1af5e2612c..20b60287be 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometProject (22) + +- CometWindowExec (21) +- CometSort (20) +- CometExchange (19) +- CometHashAggregate (18) @@ -119,20 +119,20 @@ Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNative Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(21) CometColumnarToRow [codegen id : 1] +(21) CometWindowExec Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21], [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] -(22) Window -Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] - -(23) Project [codegen id : 2] -Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +(22) CometProject Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] +Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] + +(23) CometTakeOrderedAndProject +Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#17 DESC NULLS LAST,CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST,rank_within_parent#21 ASC NULLS FIRST], output=[total_sum#16,i_category#12,i_class#13,lochierarchy#17,rank_within_parent#21]), [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21], 100, 0, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -(24) TakeOrderedAndProject +(24) CometColumnarToRow [codegen id : 1] Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt index d5ebd5a7e7..f919130b53 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -29,4 +29,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index 29e2d72920..a6b54bd89f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -1,36 +1,34 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [_w0,_w1,_w2] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] - CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt index 47ce0d4a91..1de0dfa34d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) +* CometColumnarToRow (29) ++- CometTakeOrderedAndProject (28) + +- CometProject (27) + +- CometFilter (26) + +- CometWindowExec (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_ Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 1] +(25) CometWindowExec Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] +Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22], [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] -(26) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] - -(27) Filter [codegen id : 2] +(26) CometFilter Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END -(28) Project [codegen id : 2] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] +(27) CometProject Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] +Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] + +(28) CometTakeOrderedAndProject +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST,s_store_name#17 ASC NULLS FIRST], output=[i_category#7,i_class#6,i_brand#5,s_store_name#17,s_company_name#18,d_moy#15,sum_sales#20,avg_monthly_sales#22]), [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22], 100, 0, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -(29) TakeOrderedAndProject +(29) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt index 07ad98fa22..8adffb63e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt index 61bfd1d960..682d63a75b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt @@ -1,41 +1,39 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (2) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + CometProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + CometFilter [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] + CometWindowExec [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index 47ce0d4a91..1de0dfa34d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -TakeOrderedAndProject (29) -+- * Project (28) - +- * Filter (27) - +- Window (26) - +- * CometColumnarToRow (25) +* CometColumnarToRow (29) ++- CometTakeOrderedAndProject (28) + +- CometProject (27) + +- CometFilter (26) + +- CometWindowExec (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_ Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 1] +(25) CometWindowExec Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] +Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22], [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] -(26) Window -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] - -(27) Filter [codegen id : 2] +(26) CometFilter Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END -(28) Project [codegen id : 2] -Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] +(27) CometProject Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] +Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] + +(28) CometTakeOrderedAndProject +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST,s_store_name#17 ASC NULLS FIRST], output=[i_category#7,i_class#6,i_brand#5,s_store_name#17,s_company_name#18,d_moy#15,sum_sales#20,avg_monthly_sales#22]), [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22], 100, 0, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -(29) TakeOrderedAndProject +(29) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt index 07ad98fa22..8adffb63e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt @@ -1,8 +1,8 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Filter - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index 61bfd1d960..682d63a75b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -1,41 +1,39 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] - WholeStageCodegen (2) - Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + CometProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + CometFilter [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] + CometWindowExec [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt index 4cc725ef2b..20a3805afe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt @@ -1,30 +1,29 @@ == Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (25) ++- CometProject (24) + +- CometSort (23) + +- CometExchange (22) + +- CometProject (21) + +- CometWindowExec (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -117,61 +116,58 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +(21) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(23) CometColumnarExchange +(22) CometExchange Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(24) CometSort +(23) CometSort Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] -(25) CometProject +(24) CometProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -(26) CometColumnarToRow [codegen id : 3] +(25) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(29) CometProject +(28) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(31) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt index 8a83f62759..c84faeb01d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt @@ -1,34 +1,33 @@ CometColumnarToRow +- CometProject +- CometSort - +- CometColumnarExchange - +- Project - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt index af05a7d8c6..6f23d4be47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt @@ -1,40 +1,35 @@ -WholeStageCodegen (3) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index 4cc725ef2b..20a3805afe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -1,30 +1,29 @@ == Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (25) ++- CometProject (24) + +- CometSort (23) + +- CometExchange (22) + +- CometProject (21) + +- CometWindowExec (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -117,61 +116,58 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] +(21) CometProject Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] +Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(23) CometColumnarExchange +(22) CometExchange Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(24) CometSort +(23) CometSort Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] -(25) CometProject +(24) CometProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -(26) CometColumnarToRow [codegen id : 3] +(25) CometColumnarToRow [codegen id : 1] Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(29) CometProject +(28) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(31) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt index 8a83f62759..c84faeb01d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt @@ -1,34 +1,33 @@ CometColumnarToRow +- CometProject +- CometSort - +- CometColumnarExchange - +- Project - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index af05a7d8c6..6f23d4be47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -1,40 +1,35 @@ -WholeStageCodegen (3) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] + CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt index 5737af1936..55482c8d8d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometProject (22) + +- CometWindowExec (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +(22) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] + +(23) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(24) TakeOrderedAndProject +(24) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt index 72c5bf3a04..8b95fbaf2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS _w0#9)] @@ -29,4 +29,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt index 95364332b9..8a81ec996c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt index 5737af1936..55482c8d8d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometProject (22) + +- CometWindowExec (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +(22) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] + +(23) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(24) TakeOrderedAndProject +(24) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt index 72c5bf3a04..8b95fbaf2c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS _w0#9)] @@ -29,4 +29,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt index 95364332b9..8a81ec996c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt index 78aa0a8ea8..a11e8312cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometProject (22) + +- CometWindowExec (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +(22) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] + +(23) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(24) TakeOrderedAndProject +(24) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt index 750358a202..7f6cd6b5e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS _w0#9)] @@ -29,4 +29,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt index 1b6a5f13f3..390dd3ad8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt index 78aa0a8ea8..a11e8312cf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (24) -+- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) +* CometColumnarToRow (24) ++- CometTakeOrderedAndProject (23) + +- CometProject (22) + +- CometWindowExec (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +(22) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] + +(23) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(24) TakeOrderedAndProject +(24) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt index 750358a202..7f6cd6b5e3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS _w0#9)] @@ -29,4 +29,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt index 1b6a5f13f3..390dd3ad8c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt @@ -1,38 +1,36 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (3) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt index b8a2a2bb8d..58eeeab0ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) +* CometColumnarToRow (48) ++- CometTakeOrderedAndProject (47) + +- CometProject (46) + +- CometWindowExec (45) +- CometSort (44) +- CometExchange (43) +- CometHashAggregate (42) @@ -258,20 +258,20 @@ Arguments: hashpartitioning(lochierarchy#25, _w0#75, 5), ENSURE_REQUIREMENTS, Co Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75], [lochierarchy#25 ASC NULLS FIRST, _w0#75 ASC NULLS FIRST, gross_margin#20 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 7] +(45) CometWindowExec Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] +Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76], [rank(gross_margin#20) windowspecdefinition(lochierarchy#25, _w0#75, gross_margin#20 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#76], [lochierarchy#25, _w0#75], [gross_margin#20 ASC NULLS FIRST] -(46) Window -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] -Arguments: [rank(gross_margin#20) windowspecdefinition(lochierarchy#25, _w0#75, gross_margin#20 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#76], [lochierarchy#25, _w0#75], [gross_margin#20 ASC NULLS FIRST] - -(47) Project [codegen id : 8] -Output [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +(46) CometProject Input [6]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76] +Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] + +(47) CometTakeOrderedAndProject +Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#25 DESC NULLS LAST,CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST,rank_within_parent#76 ASC NULLS FIRST], output=[gross_margin#20,i_category#21,i_class#22,lochierarchy#25,rank_within_parent#76]), [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76], 100, 0, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST, rank_within_parent#76 ASC NULLS FIRST], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] -(48) TakeOrderedAndProject +(48) CometColumnarToRow [codegen id : 7] Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] -Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST, rank_within_parent#76 ASC NULLS FIRST], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt index 6d197d1473..197fa38310 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -107,4 +107,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 81 out of 99 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 84 out of 99 eligible operators (84%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt index b799504154..d78d4c6456 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt @@ -1,72 +1,70 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (8) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (6) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (3) - HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #8 - WholeStageCodegen (2) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (5) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (4) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 +WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,rank_within_parent] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + WholeStageCodegen (6) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (1) + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class] #3 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (3) + HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #8 + WholeStageCodegen (2) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 + WholeStageCodegen (5) + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #9 + WholeStageCodegen (4) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt index b8a2a2bb8d..58eeeab0ef 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- Window (46) - +- * CometColumnarToRow (45) +* CometColumnarToRow (48) ++- CometTakeOrderedAndProject (47) + +- CometProject (46) + +- CometWindowExec (45) +- CometSort (44) +- CometExchange (43) +- CometHashAggregate (42) @@ -258,20 +258,20 @@ Arguments: hashpartitioning(lochierarchy#25, _w0#75, 5), ENSURE_REQUIREMENTS, Co Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75], [lochierarchy#25 ASC NULLS FIRST, _w0#75 ASC NULLS FIRST, gross_margin#20 ASC NULLS FIRST] -(45) CometColumnarToRow [codegen id : 7] +(45) CometWindowExec Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] +Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76], [rank(gross_margin#20) windowspecdefinition(lochierarchy#25, _w0#75, gross_margin#20 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#76], [lochierarchy#25, _w0#75], [gross_margin#20 ASC NULLS FIRST] -(46) Window -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] -Arguments: [rank(gross_margin#20) windowspecdefinition(lochierarchy#25, _w0#75, gross_margin#20 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#76], [lochierarchy#25, _w0#75], [gross_margin#20 ASC NULLS FIRST] - -(47) Project [codegen id : 8] -Output [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +(46) CometProject Input [6]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76] +Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] + +(47) CometTakeOrderedAndProject +Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#25 DESC NULLS LAST,CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST,rank_within_parent#76 ASC NULLS FIRST], output=[gross_margin#20,i_category#21,i_class#22,lochierarchy#25,rank_within_parent#76]), [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76], 100, 0, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST, rank_within_parent#76 ASC NULLS FIRST], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] -(48) TakeOrderedAndProject +(48) CometColumnarToRow [codegen id : 7] Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] -Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST, rank_within_parent#76 ASC NULLS FIRST], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt index 6d197d1473..197fa38310 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -107,4 +107,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 81 out of 99 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 84 out of 99 eligible operators (84%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt index b799504154..d78d4c6456 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt @@ -1,72 +1,70 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (8) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (6) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (3) - HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #8 - WholeStageCodegen (2) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (5) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (4) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 +WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,rank_within_parent] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + WholeStageCodegen (6) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (1) + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class] #3 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (3) + HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #8 + WholeStageCodegen (2) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 + WholeStageCodegen (5) + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #9 + WholeStageCodegen (4) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt index b72fd62948..8b6d6fe277 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt @@ -1,54 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (29) + : : +- CometFilter (28) + : : +- CometWindowExec (27) + : : +- CometFilter (26) + : : +- CometWindowExec (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- CometBroadcastExchange (37) + : +- CometProject (36) + : +- CometWindowExec (35) + : +- CometSort (34) + : +- CometColumnarExchange (33) + : +- * HashAggregate (32) + : +- * CometColumnarToRow (31) + : +- ReusedExchange (30) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -166,136 +160,112 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 2] +(25) CometWindowExec Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(27) Filter [codegen id : 3] +(26) CometFilter Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(28) Window +(27) CometWindowExec Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22], [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(29) Filter [codegen id : 10] +(28) CometFilter Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(30) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] +(29) CometProject Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -(31) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(32) CometColumnarToRow [codegen id : 4] +(31) CometColumnarToRow [codegen id : 2] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(33) HashAggregate [codegen id : 4] +(32) HashAggregate [codegen id : 2] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] -(34) CometColumnarExchange +(33) CometColumnarExchange Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(35) CometSort +(34) CometSort Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] - -(37) Window +(35) CometWindowExec Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32], [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(36) CometProject Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(39) BroadcastExchange +(37) CometBroadcastExchange Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] +Right output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)], Inner, BuildRight -(41) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] +(39) CometProject Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -(42) ReusedExchange [Reuses operator id: 34] -Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] - -(43) CometSort -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 8] -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] - -(45) Window -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] - -(46) Project [codegen id : 9] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] +(40) ReusedExchange [Reuses operator id: 37] +Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -(47) BroadcastExchange -Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] +Right output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#38 - 1)], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 1)] -Join type: Inner -Join condition: None +(42) CometProject +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +Arguments: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#37 AS nsum#40] -(49) Project [codegen id : 10] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(43) CometTakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -(50) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +(44) CometColumnarToRow [codegen id : 4] +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(53) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) BroadcastExchange +(48) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt index 03f1eab564..eeae7c1e95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt @@ -1,80 +1,78 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] @@ -105,4 +103,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt index b08d56b5ca..77ba154cab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt @@ -1,77 +1,57 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (10) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (3) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) +WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] CometColumnarToRow InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt index b72fd62948..8b6d6fe277 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt @@ -1,54 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (29) + : : +- CometFilter (28) + : : +- CometWindowExec (27) + : : +- CometFilter (26) + : : +- CometWindowExec (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- CometBroadcastExchange (37) + : +- CometProject (36) + : +- CometWindowExec (35) + : +- CometSort (34) + : +- CometColumnarExchange (33) + : +- * HashAggregate (32) + : +- * CometColumnarToRow (31) + : +- ReusedExchange (30) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -166,136 +160,112 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 2] +(25) CometWindowExec Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(27) Filter [codegen id : 3] +(26) CometFilter Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(28) Window +(27) CometWindowExec Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22], [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(29) Filter [codegen id : 10] +(28) CometFilter Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(30) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] +(29) CometProject Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -(31) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(32) CometColumnarToRow [codegen id : 4] +(31) CometColumnarToRow [codegen id : 2] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(33) HashAggregate [codegen id : 4] +(32) HashAggregate [codegen id : 2] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] -(34) CometColumnarExchange +(33) CometColumnarExchange Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(35) CometSort +(34) CometSort Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] - -(37) Window +(35) CometWindowExec Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32], [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +(36) CometProject Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(39) BroadcastExchange +(37) CometBroadcastExchange Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] +Right output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)], Inner, BuildRight -(41) Project [codegen id : 10] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] +(39) CometProject Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -(42) ReusedExchange [Reuses operator id: 34] -Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] - -(43) CometSort -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 8] -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] - -(45) Window -Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] -Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] - -(46) Project [codegen id : 9] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] +(40) ReusedExchange [Reuses operator id: 37] +Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -(47) BroadcastExchange -Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] +Right output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#38 - 1)], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 10] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] -Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 1)] -Join type: Inner -Join condition: None +(42) CometProject +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +Arguments: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#37 AS nsum#40] -(49) Project [codegen id : 10] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +(43) CometTakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -(50) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] -Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +(44) CometColumnarToRow [codegen id : 4] +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(53) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) BroadcastExchange +(48) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt index 03f1eab564..eeae7c1e95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt @@ -1,80 +1,78 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] @@ -105,4 +103,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt index b08d56b5ca..77ba154cab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt @@ -1,77 +1,57 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (10) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (3) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (2) +WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] CometColumnarToRow InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt index 74702d596d..2c993b48e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt @@ -1,82 +1,79 @@ == Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) +* CometColumnarToRow (75) ++- CometTakeOrderedAndProject (74) + +- CometHashAggregate (73) + +- CometExchange (72) + +- CometHashAggregate (71) + +- CometUnion (70) + :- CometProject (25) + : +- CometFilter (24) + : +- CometWindowExec (23) + : +- CometSort (22) + : +- CometWindowExec (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- CometProject (47) + : +- CometFilter (46) + : +- CometWindowExec (45) + : +- CometSort (44) + : +- CometWindowExec (43) + : +- CometSort (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometExchange (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometBroadcastExchange (29) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (35) + +- CometProject (69) + +- CometFilter (68) + +- CometWindowExec (67) + +- CometSort (66) + +- CometWindowExec (65) + +- CometSort (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometProject (56) + : +- CometBroadcastHashJoin (55) + : :- CometBroadcastExchange (51) + : : +- CometProject (50) + : : +- CometFilter (49) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) + +- ReusedExchange (57) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -173,30 +170,27 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 1] +(21) CometWindowExec Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] +(22) CometSort Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] -(24) Window +(23) CometWindowExec Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(25) Filter [codegen id : 3] +(24) CometFilter Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(25) CometProject Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -204,100 +198,97 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(29) CometProject +(28) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometBroadcastExchange +(29) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(33) CometProject +(32) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(34) CometBroadcastHashJoin +(33) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(35) CometProject +(34) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(36) ReusedExchange [Reuses operator id: 13] +(35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(37) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(38) CometProject +(37) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(39) CometHashAggregate +(38) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(40) CometExchange +(39) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(41) CometHashAggregate +(40) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(42) CometExchange +(41) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort +(42) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window +(43) CometWindowExec Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(46) Sort [codegen id : 5] +(44) CometSort Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] -(47) Window +(45) CometWindowExec Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(48) Filter [codegen id : 6] +(46) CometFilter Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +(47) CometProject Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -305,158 +296,156 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter +(49) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(52) CometProject +(50) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(53) CometBroadcastExchange +(51) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(56) CometProject +(54) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(57) CometBroadcastHashJoin +(55) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(58) CometProject +(56) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(59) ReusedExchange [Reuses operator id: 13] +(57) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(60) CometBroadcastHashJoin +(58) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(61) CometProject +(59) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(62) CometHashAggregate +(60) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(61) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(64) CometHashAggregate +(62) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(65) CometExchange +(63) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) CometSort +(64) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) CometColumnarToRow [codegen id : 7] +(65) CometWindowExec Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] +(66) CometSort Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] -(70) Window +(67) CometWindowExec Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(71) Filter [codegen id : 9] +(68) CometFilter Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(69) CometProject Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(73) Union +(70) CometUnion +Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(74) HashAggregate [codegen id : 10] +(71) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarExchange +(72) CometExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(76) CometHashAggregate +(73) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(77) CometTakeOrderedAndProject +(74) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(78) CometColumnarToRow [codegen id : 11] +(75) CometColumnarToRow [codegen id : 1] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter +(77) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(81) CometProject +(78) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(79) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(83) BroadcastExchange +(80) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt index 4988a26564..6a8de7fe28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt @@ -1,94 +1,91 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt index ca80833ee5..4f128e83de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt @@ -1,110 +1,87 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometUnion [channel,item,return_ratio,return_rank,currency_rank] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt index 74702d596d..2c993b48e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt @@ -1,82 +1,79 @@ == Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) +* CometColumnarToRow (75) ++- CometTakeOrderedAndProject (74) + +- CometHashAggregate (73) + +- CometExchange (72) + +- CometHashAggregate (71) + +- CometUnion (70) + :- CometProject (25) + : +- CometFilter (24) + : +- CometWindowExec (23) + : +- CometSort (22) + : +- CometWindowExec (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- CometProject (47) + : +- CometFilter (46) + : +- CometWindowExec (45) + : +- CometSort (44) + : +- CometWindowExec (43) + : +- CometSort (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometExchange (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometBroadcastExchange (29) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (35) + +- CometProject (69) + +- CometFilter (68) + +- CometWindowExec (67) + +- CometSort (66) + +- CometWindowExec (65) + +- CometSort (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometProject (56) + : +- CometBroadcastHashJoin (55) + : :- CometBroadcastExchange (51) + : : +- CometProject (50) + : : +- CometFilter (49) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) + +- ReusedExchange (57) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -173,30 +170,27 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 1] +(21) CometWindowExec Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] +(22) CometSort Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] -(24) Window +(23) CometWindowExec Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(25) Filter [codegen id : 3] +(24) CometFilter Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(25) CometProject Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -204,100 +198,97 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(29) CometProject +(28) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometBroadcastExchange +(29) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(33) CometProject +(32) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(34) CometBroadcastHashJoin +(33) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(35) CometProject +(34) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(36) ReusedExchange [Reuses operator id: 13] +(35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(37) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(38) CometProject +(37) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(39) CometHashAggregate +(38) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(40) CometExchange +(39) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(41) CometHashAggregate +(40) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(42) CometExchange +(41) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort +(42) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window +(43) CometWindowExec Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(46) Sort [codegen id : 5] +(44) CometSort Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] -(47) Window +(45) CometWindowExec Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(48) Filter [codegen id : 6] +(46) CometFilter Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +(47) CometProject Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -305,158 +296,156 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter +(49) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(52) CometProject +(50) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(53) CometBroadcastExchange +(51) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(56) CometProject +(54) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(57) CometBroadcastHashJoin +(55) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(58) CometProject +(56) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(59) ReusedExchange [Reuses operator id: 13] +(57) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(60) CometBroadcastHashJoin +(58) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(61) CometProject +(59) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(62) CometHashAggregate +(60) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(61) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(64) CometHashAggregate +(62) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(65) CometExchange +(63) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) CometSort +(64) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) CometColumnarToRow [codegen id : 7] +(65) CometWindowExec Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] +(66) CometSort Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] -(70) Window +(67) CometWindowExec Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(71) Filter [codegen id : 9] +(68) CometFilter Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(69) CometProject Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(73) Union +(70) CometUnion +Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(74) HashAggregate [codegen id : 10] +(71) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarExchange +(72) CometExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(76) CometHashAggregate +(73) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(77) CometTakeOrderedAndProject +(74) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(78) CometColumnarToRow [codegen id : 11] +(75) CometColumnarToRow [codegen id : 1] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter +(77) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(81) CometProject +(78) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(79) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(83) BroadcastExchange +(80) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt index 4988a26564..6a8de7fe28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt @@ -1,94 +1,91 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt index ca80833ee5..4f128e83de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt @@ -1,110 +1,87 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometUnion [channel,item,return_ratio,return_rank,currency_rank] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt index d850c164a2..000038216f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt @@ -1,82 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (78) -+- * Filter (77) - +- * HashAggregate (76) - +- * HashAggregate (75) - +- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- Window (66) - : +- * CometColumnarToRow (65) - : +- CometSort (64) - : +- CometExchange (63) - : +- CometProject (62) - : +- CometFilter (61) - : +- CometSortMergeJoin (60) - : :- CometSort (31) - : : +- CometColumnarExchange (30) - : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometColumnarExchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (17) - : : : +- Window (16) - : : : +- * CometColumnarToRow (15) - : : : +- CometSort (14) - : : : +- CometColumnarExchange (13) - : : : +- * HashAggregate (12) - : : : +- * CometColumnarToRow (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- Window (21) - : : +- * CometColumnarToRow (20) - : : +- CometSort (19) - : : +- ReusedExchange (18) - : +- CometSort (59) - : +- CometColumnarExchange (58) - : +- * HashAggregate (57) - : +- * CometColumnarToRow (56) - : +- CometColumnarExchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) - : :- * Project (45) - : : +- Window (44) - : : +- * CometColumnarToRow (43) - : : +- CometSort (42) - : : +- CometColumnarExchange (41) - : : +- * HashAggregate (40) - : : +- * CometColumnarToRow (39) - : : +- CometExchange (38) - : : +- CometHashAggregate (37) - : : +- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (33) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (32) - : : +- ReusedExchange (34) - : +- BroadcastExchange (51) - : +- * Project (50) - : +- Window (49) - : +- * CometColumnarToRow (48) - : +- CometSort (47) - : +- ReusedExchange (46) - +- BroadcastExchange (72) - +- * Project (71) - +- Window (70) - +- * CometColumnarToRow (69) - +- CometSort (68) - +- ReusedExchange (67) +* CometColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometFilter (69) + +- CometHashAggregate (68) + +- CometHashAggregate (67) + +- CometProject (66) + +- CometBroadcastHashJoin (65) + :- CometWindowExec (59) + : +- CometSort (58) + : +- CometExchange (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometSortMergeJoin (54) + : :- CometSort (28) + : : +- CometExchange (27) + : : +- CometHashAggregate (26) + : : +- CometExchange (25) + : : +- CometHashAggregate (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (16) + : : : +- CometWindowExec (15) + : : : +- CometSort (14) + : : : +- CometColumnarExchange (13) + : : : +- * HashAggregate (12) + : : : +- * CometColumnarToRow (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (21) + : : +- CometProject (20) + : : +- CometWindowExec (19) + : : +- CometSort (18) + : : +- ReusedExchange (17) + : +- CometSort (53) + : +- CometExchange (52) + : +- CometHashAggregate (51) + : +- CometExchange (50) + : +- CometHashAggregate (49) + : +- CometProject (48) + : +- CometBroadcastHashJoin (47) + : :- CometProject (41) + : : +- CometWindowExec (40) + : : +- CometSort (39) + : : +- CometColumnarExchange (38) + : : +- * HashAggregate (37) + : : +- * CometColumnarToRow (36) + : : +- CometExchange (35) + : : +- CometHashAggregate (34) + : : +- CometProject (33) + : : +- CometBroadcastHashJoin (32) + : : :- CometFilter (30) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (29) + : : +- ReusedExchange (31) + : +- CometBroadcastExchange (46) + : +- CometProject (45) + : +- CometWindowExec (44) + : +- CometSort (43) + : +- ReusedExchange (42) + +- CometBroadcastExchange (64) + +- CometProject (63) + +- CometWindowExec (62) + +- CometSort (61) + +- ReusedExchange (60) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -146,315 +139,279 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) CometColumnarToRow [codegen id : 2] +(15) CometWindowExec Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12], [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(16) Window -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(17) Project [codegen id : 6] -Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] +(16) CometProject Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] +Arguments: [item_sk#10, d_date#6, sumws#11, rk#12], [item_sk#10, d_date#6, sumws#11, rk#12] -(18) ReusedExchange [Reuses operator id: 13] +(17) ReusedExchange [Reuses operator id: 13] Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -(19) CometSort +(18) CometSort Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16], [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] - -(21) Window +(19) CometWindowExec Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] +Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17], [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] -(22) Project [codegen id : 5] -Output [3]: [item_sk#13, sumws#15, rk#17] +(20) CometProject Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] +Arguments: [item_sk#13, sumws#15, rk#17], [item_sk#13, sumws#15, rk#17] -(23) BroadcastExchange +(21) CometBroadcastExchange Input [3]: [item_sk#13, sumws#15, rk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [item_sk#13, sumws#15, rk#17] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#13] -Join type: Inner -Join condition: (rk#12 >= rk#17) +(22) CometBroadcastHashJoin +Left output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] +Right output [3]: [item_sk#13, sumws#15, rk#17] +Arguments: [item_sk#10], [item_sk#13], Inner, (rk#12 >= rk#17), BuildRight -(25) Project [codegen id : 6] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] +(23) CometProject Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] +Arguments: [item_sk#10, d_date#6, sumws#11, sumws#15], [item_sk#10, d_date#6, sumws#11, sumws#15] -(26) HashAggregate [codegen id : 6] +(24) CometHashAggregate Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [partial_sum(sumws#15)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(27) CometColumnarExchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(25) CometExchange +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#18, isEmpty#19] +Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 7] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(29) HashAggregate [codegen id : 7] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +(26) CometHashAggregate +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#18, isEmpty#19] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [sum(sumws#15)] -Aggregate Attributes [1]: [sum(sumws#15)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#15)#22 AS cume_sales#23] -(30) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(27) CometExchange +Input [3]: [item_sk#10, d_date#6, cume_sales#20] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(31) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(28) CometSort +Input [3]: [item_sk#10, d_date#6, cume_sales#20] +Arguments: [item_sk#10, d_date#6, cume_sales#20], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(33) CometFilter -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) - -(34) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#28, d_date#29] - -(35) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight - -(36) CometProject -Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] -Arguments: [ss_item_sk#24, ss_sales_price#25, d_date#29], [ss_item_sk#24, ss_sales_price#25, d_date#29] - -(37) CometHashAggregate -Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] - -(38) CometExchange -Input [3]: [ss_item_sk#24, d_date#29, sum#30] -Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_item_sk#24, d_date#29, sum#30] - -(40) HashAggregate [codegen id : 8] -Input [3]: [ss_item_sk#24, d_date#29, sum#30] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] -Results [4]: [ss_item_sk#24 AS item_sk#32, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS sumss#33, ss_item_sk#24] - -(41) CometColumnarExchange -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(42) CometSort -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -Arguments: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] - -(43) CometColumnarToRow [codegen id : 9] -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +(30) CometFilter +Input [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_item_sk#21) + +(31) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#25, d_date#26] + +(32) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] +Right output [2]: [d_date_sk#25, d_date#26] +Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight + +(33) CometProject +Input [5]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23, d_date_sk#25, d_date#26] +Arguments: [ss_item_sk#21, ss_sales_price#22, d_date#26], [ss_item_sk#21, ss_sales_price#22, d_date#26] + +(34) CometHashAggregate +Input [3]: [ss_item_sk#21, ss_sales_price#22, d_date#26] +Keys [2]: [ss_item_sk#21, d_date#26] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#22))] + +(35) CometExchange +Input [3]: [ss_item_sk#21, d_date#26, sum#27] +Arguments: hashpartitioning(ss_item_sk#21, d_date#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(36) CometColumnarToRow [codegen id : 3] +Input [3]: [ss_item_sk#21, d_date#26, sum#27] + +(37) HashAggregate [codegen id : 3] +Input [3]: [ss_item_sk#21, d_date#26, sum#27] +Keys [2]: [ss_item_sk#21, d_date#26] +Functions [1]: [sum(UnscaledValue(ss_sales_price#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#22))#28] +Results [4]: [ss_item_sk#21 AS item_sk#29, d_date#26, MakeDecimal(sum(UnscaledValue(ss_sales_price#22))#28,17,2) AS sumss#30, ss_item_sk#21] + +(38) CometColumnarExchange +Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] +Arguments: hashpartitioning(ss_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(39) CometSort +Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] +Arguments: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21], [ss_item_sk#21 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST] + +(40) CometWindowExec +Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] +Arguments: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21, rk#31], [row_number() windowspecdefinition(ss_item_sk#21, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#31], [ss_item_sk#21], [d_date#26 ASC NULLS FIRST] + +(41) CometProject +Input [5]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21, rk#31] +Arguments: [item_sk#29, d_date#26, sumss#30, rk#31], [item_sk#29, d_date#26, sumss#30, rk#31] + +(42) ReusedExchange [Reuses operator id: 38] +Output [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] + +(43) CometSort +Input [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] +Arguments: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35], [ss_item_sk#35 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] + +(44) CometWindowExec +Input [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] +Arguments: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35, rk#36], [row_number() windowspecdefinition(ss_item_sk#35, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [ss_item_sk#35], [d_date#33 ASC NULLS FIRST] + +(45) CometProject +Input [5]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35, rk#36] +Arguments: [item_sk#32, sumss#34, rk#36], [item_sk#32, sumss#34, rk#36] + +(46) CometBroadcastExchange +Input [3]: [item_sk#32, sumss#34, rk#36] +Arguments: [item_sk#32, sumss#34, rk#36] + +(47) CometBroadcastHashJoin +Left output [4]: [item_sk#29, d_date#26, sumss#30, rk#31] +Right output [3]: [item_sk#32, sumss#34, rk#36] +Arguments: [item_sk#29], [item_sk#32], Inner, (rk#31 >= rk#36), BuildRight + +(48) CometProject +Input [7]: [item_sk#29, d_date#26, sumss#30, rk#31, item_sk#32, sumss#34, rk#36] +Arguments: [item_sk#29, d_date#26, sumss#30, sumss#34], [item_sk#29, d_date#26, sumss#30, sumss#34] + +(49) CometHashAggregate +Input [4]: [item_sk#29, d_date#26, sumss#30, sumss#34] +Keys [3]: [item_sk#29, d_date#26, sumss#30] +Functions [1]: [partial_sum(sumss#34)] + +(50) CometExchange +Input [5]: [item_sk#29, d_date#26, sumss#30, sum#37, isEmpty#38] +Arguments: hashpartitioning(item_sk#29, d_date#26, sumss#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(51) CometHashAggregate +Input [5]: [item_sk#29, d_date#26, sumss#30, sum#37, isEmpty#38] +Keys [3]: [item_sk#29, d_date#26, sumss#30] +Functions [1]: [sum(sumss#34)] + +(52) CometExchange +Input [3]: [item_sk#29, d_date#26, cume_sales#39] +Arguments: hashpartitioning(item_sk#29, d_date#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(53) CometSort +Input [3]: [item_sk#29, d_date#26, cume_sales#39] +Arguments: [item_sk#29, d_date#26, cume_sales#39], [item_sk#29 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST] + +(54) CometSortMergeJoin +Left output [3]: [item_sk#10, d_date#6, cume_sales#20] +Right output [3]: [item_sk#29, d_date#26, cume_sales#39] +Arguments: [item_sk#10, d_date#6], [item_sk#29, d_date#26], FullOuter + +(55) CometFilter +Input [6]: [item_sk#10, d_date#6, cume_sales#20, item_sk#29, d_date#26, cume_sales#39] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#29 END) + +(56) CometProject +Input [6]: [item_sk#10, d_date#6, cume_sales#20, item_sk#29, d_date#26, cume_sales#39] +Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#29 END AS item_sk#40, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#26 END AS d_date#41, cume_sales#20 AS web_sales#42, cume_sales#39 AS store_sales#43] + +(57) CometExchange +Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] +Arguments: hashpartitioning(item_sk#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] + +(58) CometSort +Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] +Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43], [item_sk#40 ASC NULLS FIRST, d_date#41 ASC NULLS FIRST] + +(59) CometWindowExec +Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] +Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44], [row_number() windowspecdefinition(item_sk#40, d_date#41 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#44], [item_sk#40], [d_date#41 ASC NULLS FIRST] + +(60) ReusedExchange [Reuses operator id: 57] +Output [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] + +(61) CometSort +Input [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] +Arguments: [item_sk#45, d_date#46, web_sales#47, store_sales#48], [item_sk#45 ASC NULLS FIRST, d_date#46 ASC NULLS FIRST] + +(62) CometWindowExec +Input [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] +Arguments: [item_sk#45, d_date#46, web_sales#47, store_sales#48, rk#49], [row_number() windowspecdefinition(item_sk#45, d_date#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#45], [d_date#46 ASC NULLS FIRST] -(44) Window -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] +(63) CometProject +Input [5]: [item_sk#45, d_date#46, web_sales#47, store_sales#48, rk#49] +Arguments: [item_sk#45, web_sales#47, store_sales#48, rk#49], [item_sk#45, web_sales#47, store_sales#48, rk#49] -(45) Project [codegen id : 13] -Output [4]: [item_sk#32, d_date#29, sumss#33, rk#34] -Input [5]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24, rk#34] +(64) CometBroadcastExchange +Input [4]: [item_sk#45, web_sales#47, store_sales#48, rk#49] +Arguments: [item_sk#45, web_sales#47, store_sales#48, rk#49] -(46) ReusedExchange [Reuses operator id: 41] -Output [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +(65) CometBroadcastHashJoin +Left output [5]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44] +Right output [4]: [item_sk#45, web_sales#47, store_sales#48, rk#49] +Arguments: [item_sk#40], [item_sk#45], Inner, (rk#44 >= rk#49), BuildRight -(47) CometSort -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -Arguments: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38], [ss_item_sk#38 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] +(66) CometProject +Input [9]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44, item_sk#45, web_sales#47, store_sales#48, rk#49] +Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48], [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48] -(48) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] - -(49) Window -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -Arguments: [row_number() windowspecdefinition(ss_item_sk#38, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#39], [ss_item_sk#38], [d_date#36 ASC NULLS FIRST] - -(50) Project [codegen id : 12] -Output [3]: [item_sk#35, sumss#37, rk#39] -Input [5]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38, rk#39] - -(51) BroadcastExchange -Input [3]: [item_sk#35, sumss#37, rk#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [item_sk#32] -Right keys [1]: [item_sk#35] -Join type: Inner -Join condition: (rk#34 >= rk#39) - -(53) Project [codegen id : 13] -Output [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] -Input [7]: [item_sk#32, d_date#29, sumss#33, rk#34, item_sk#35, sumss#37, rk#39] - -(54) HashAggregate [codegen id : 13] -Input [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] -Keys [3]: [item_sk#32, d_date#29, sumss#33] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] - -(55) CometColumnarExchange -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] -Arguments: hashpartitioning(item_sk#32, d_date#29, sumss#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(56) CometColumnarToRow [codegen id : 14] -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] - -(57) HashAggregate [codegen id : 14] -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] -Keys [3]: [item_sk#32, d_date#29, sumss#33] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#44] -Results [3]: [item_sk#32, d_date#29, sum(sumss#37)#44 AS cume_sales#45] - -(58) CometColumnarExchange -Input [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: hashpartitioning(item_sk#32, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(59) CometSort -Input [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#32, d_date#29, cume_sales#45], [item_sk#32 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] - -(60) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#23] -Right output [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#10, d_date#6], [item_sk#32, d_date#29], FullOuter - -(61) CometFilter -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END) - -(62) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END AS item_sk#46, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#47, cume_sales#23 AS web_sales#48, cume_sales#45 AS store_sales#49] - -(63) CometExchange -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(64) CometSort -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] - -(65) CometColumnarToRow [codegen id : 15] -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] - -(66) Window -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 ASC NULLS FIRST] - -(67) ReusedExchange [Reuses operator id: 63] -Output [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] - -(68) CometSort -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] -Arguments: [item_sk#51, d_date#52, web_sales#53, store_sales#54], [item_sk#51 ASC NULLS FIRST, d_date#52 ASC NULLS FIRST] - -(69) CometColumnarToRow [codegen id : 30] -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] - -(70) Window -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] -Arguments: [row_number() windowspecdefinition(item_sk#51, d_date#52 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#55], [item_sk#51], [d_date#52 ASC NULLS FIRST] - -(71) Project [codegen id : 31] -Output [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] -Input [5]: [item_sk#51, d_date#52, web_sales#53, store_sales#54, rk#55] - -(72) BroadcastExchange -Input [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(73) BroadcastHashJoin [codegen id : 32] -Left keys [1]: [item_sk#46] -Right keys [1]: [item_sk#51] -Join type: Inner -Join condition: (rk#50 >= rk#55) - -(74) Project [codegen id : 32] -Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#51, web_sales#53, store_sales#54, rk#55] - -(75) HashAggregate [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] -Aggregate Attributes [2]: [max#56, max#57] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] - -(76) HashAggregate [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [max(web_sales#53), max(store_sales#54)] -Aggregate Attributes [2]: [max(web_sales#53)#60, max(store_sales#54)#61] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#60 AS web_cumulative#62, max(store_sales#54)#61 AS store_cumulative#63] - -(77) Filter [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] -Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) - -(78) TakeOrderedAndProject -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] -Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +(67) CometHashAggregate +Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48] +Keys [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] +Functions [2]: [partial_max(web_sales#47), partial_max(store_sales#48)] + +(68) CometHashAggregate +Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, max#50, max#51] +Keys [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] +Functions [2]: [max(web_sales#47), max(store_sales#48)] + +(69) CometFilter +Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] +Condition : ((isnotnull(web_cumulative#52) AND isnotnull(store_cumulative#53)) AND (web_cumulative#52 > store_cumulative#53)) + +(70) CometTakeOrderedAndProject +Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#40 ASC NULLS FIRST,d_date#41 ASC NULLS FIRST], output=[item_sk#40,d_date#41,web_sales#42,store_sales#43,web_cumulative#52,store_cumulative#53]), [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53], 100, 0, [item_sk#40 ASC NULLS FIRST, d_date#41 ASC NULLS FIRST], [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] + +(71) CometColumnarToRow [codegen id : 9] +Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter +(73) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(81) CometProject +(74) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(82) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(83) BroadcastExchange +(76) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt index f859e0bf3e..6e3fd36930 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt @@ -1,224 +1,211 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometWindowExec + : : : +- CometSort + : : : +- CometColumnarExchange + : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : : +- CometColumnarToRow + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 130 out of 196 eligible operators (66%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 176 out of 196 eligible operators (89%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt index c88883dbad..51d1e38ec6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt @@ -1,122 +1,86 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (32) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (7) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (6) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] +WholeStageCodegen (9) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometHashAggregate [max,max] [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative,max(web_sales),max(store_sales)] + CometHashAggregate [web_sales,store_sales] [item_sk,d_date,web_sales,store_sales,max,max] + CometProject [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + CometBroadcastHashJoin [item_sk,d_date,web_sales,store_sales,rk,item_sk,web_sales,store_sales,rk] + CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #2 + CometHashAggregate [sumws,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumws)] + CometExchange [item_sk,d_date,sumws] #3 + CometHashAggregate [sumws] [item_sk,d_date,sumws,sum,isEmpty] + CometProject [item_sk,d_date,sumws,sumws] + CometBroadcastHashJoin [item_sk,d_date,sumws,rk,item_sk,sumws,rk] + CometProject [item_sk,d_date,sumws,rk] + CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + CometColumnarToRow InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (14) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (13) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] + CometExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [item_sk,sumws,rk] #8 + CometProject [item_sk,sumws,rk] + CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #9 + CometHashAggregate [sumss,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumss)] + CometExchange [item_sk,d_date,sumss] #10 + CometHashAggregate [sumss] [item_sk,d_date,sumss,sum,isEmpty] + CometProject [item_sk,d_date,sumss,sumss] + CometBroadcastHashJoin [item_sk,d_date,sumss,rk,item_sk,sumss,rk] + CometProject [item_sk,d_date,sumss,rk] + CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #11 + WholeStageCodegen (3) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + CometColumnarToRow InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #11 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (31) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (30) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 + CometBroadcastExchange [item_sk,sumss,rk] #13 + CometProject [item_sk,sumss,rk] + CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + CometBroadcastExchange [item_sk,web_sales,store_sales,rk] #14 + CometProject [item_sk,web_sales,store_sales,rk] + CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt index d850c164a2..000038216f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt @@ -1,82 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (78) -+- * Filter (77) - +- * HashAggregate (76) - +- * HashAggregate (75) - +- * Project (74) - +- * BroadcastHashJoin Inner BuildRight (73) - :- Window (66) - : +- * CometColumnarToRow (65) - : +- CometSort (64) - : +- CometExchange (63) - : +- CometProject (62) - : +- CometFilter (61) - : +- CometSortMergeJoin (60) - : :- CometSort (31) - : : +- CometColumnarExchange (30) - : : +- * HashAggregate (29) - : : +- * CometColumnarToRow (28) - : : +- CometColumnarExchange (27) - : : +- * HashAggregate (26) - : : +- * Project (25) - : : +- * BroadcastHashJoin Inner BuildRight (24) - : : :- * Project (17) - : : : +- Window (16) - : : : +- * CometColumnarToRow (15) - : : : +- CometSort (14) - : : : +- CometColumnarExchange (13) - : : : +- * HashAggregate (12) - : : : +- * CometColumnarToRow (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (23) - : : +- * Project (22) - : : +- Window (21) - : : +- * CometColumnarToRow (20) - : : +- CometSort (19) - : : +- ReusedExchange (18) - : +- CometSort (59) - : +- CometColumnarExchange (58) - : +- * HashAggregate (57) - : +- * CometColumnarToRow (56) - : +- CometColumnarExchange (55) - : +- * HashAggregate (54) - : +- * Project (53) - : +- * BroadcastHashJoin Inner BuildRight (52) - : :- * Project (45) - : : +- Window (44) - : : +- * CometColumnarToRow (43) - : : +- CometSort (42) - : : +- CometColumnarExchange (41) - : : +- * HashAggregate (40) - : : +- * CometColumnarToRow (39) - : : +- CometExchange (38) - : : +- CometHashAggregate (37) - : : +- CometProject (36) - : : +- CometBroadcastHashJoin (35) - : : :- CometFilter (33) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (32) - : : +- ReusedExchange (34) - : +- BroadcastExchange (51) - : +- * Project (50) - : +- Window (49) - : +- * CometColumnarToRow (48) - : +- CometSort (47) - : +- ReusedExchange (46) - +- BroadcastExchange (72) - +- * Project (71) - +- Window (70) - +- * CometColumnarToRow (69) - +- CometSort (68) - +- ReusedExchange (67) +* CometColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometFilter (69) + +- CometHashAggregate (68) + +- CometHashAggregate (67) + +- CometProject (66) + +- CometBroadcastHashJoin (65) + :- CometWindowExec (59) + : +- CometSort (58) + : +- CometExchange (57) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometSortMergeJoin (54) + : :- CometSort (28) + : : +- CometExchange (27) + : : +- CometHashAggregate (26) + : : +- CometExchange (25) + : : +- CometHashAggregate (24) + : : +- CometProject (23) + : : +- CometBroadcastHashJoin (22) + : : :- CometProject (16) + : : : +- CometWindowExec (15) + : : : +- CometSort (14) + : : : +- CometColumnarExchange (13) + : : : +- * HashAggregate (12) + : : : +- * CometColumnarToRow (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (21) + : : +- CometProject (20) + : : +- CometWindowExec (19) + : : +- CometSort (18) + : : +- ReusedExchange (17) + : +- CometSort (53) + : +- CometExchange (52) + : +- CometHashAggregate (51) + : +- CometExchange (50) + : +- CometHashAggregate (49) + : +- CometProject (48) + : +- CometBroadcastHashJoin (47) + : :- CometProject (41) + : : +- CometWindowExec (40) + : : +- CometSort (39) + : : +- CometColumnarExchange (38) + : : +- * HashAggregate (37) + : : +- * CometColumnarToRow (36) + : : +- CometExchange (35) + : : +- CometHashAggregate (34) + : : +- CometProject (33) + : : +- CometBroadcastHashJoin (32) + : : :- CometFilter (30) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (29) + : : +- ReusedExchange (31) + : +- CometBroadcastExchange (46) + : +- CometProject (45) + : +- CometWindowExec (44) + : +- CometSort (43) + : +- ReusedExchange (42) + +- CometBroadcastExchange (64) + +- CometProject (63) + +- CometWindowExec (62) + +- CometSort (61) + +- ReusedExchange (60) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -146,315 +139,279 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) CometColumnarToRow [codegen id : 2] +(15) CometWindowExec Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12], [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(16) Window -Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(17) Project [codegen id : 6] -Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] +(16) CometProject Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] +Arguments: [item_sk#10, d_date#6, sumws#11, rk#12], [item_sk#10, d_date#6, sumws#11, rk#12] -(18) ReusedExchange [Reuses operator id: 13] +(17) ReusedExchange [Reuses operator id: 13] Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -(19) CometSort +(18) CometSort Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16], [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 4] -Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] - -(21) Window +(19) CometWindowExec Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] +Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17], [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] -(22) Project [codegen id : 5] -Output [3]: [item_sk#13, sumws#15, rk#17] +(20) CometProject Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] +Arguments: [item_sk#13, sumws#15, rk#17], [item_sk#13, sumws#15, rk#17] -(23) BroadcastExchange +(21) CometBroadcastExchange Input [3]: [item_sk#13, sumws#15, rk#17] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [item_sk#13, sumws#15, rk#17] -(24) BroadcastHashJoin [codegen id : 6] -Left keys [1]: [item_sk#10] -Right keys [1]: [item_sk#13] -Join type: Inner -Join condition: (rk#12 >= rk#17) +(22) CometBroadcastHashJoin +Left output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] +Right output [3]: [item_sk#13, sumws#15, rk#17] +Arguments: [item_sk#10], [item_sk#13], Inner, (rk#12 >= rk#17), BuildRight -(25) Project [codegen id : 6] -Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] +(23) CometProject Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] +Arguments: [item_sk#10, d_date#6, sumws#11, sumws#15], [item_sk#10, d_date#6, sumws#11, sumws#15] -(26) HashAggregate [codegen id : 6] +(24) CometHashAggregate Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [partial_sum(sumws#15)] -Aggregate Attributes [2]: [sum#18, isEmpty#19] -Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(27) CometColumnarExchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(25) CometExchange +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#18, isEmpty#19] +Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(28) CometColumnarToRow [codegen id : 7] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] - -(29) HashAggregate [codegen id : 7] -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +(26) CometHashAggregate +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#18, isEmpty#19] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [sum(sumws#15)] -Aggregate Attributes [1]: [sum(sumws#15)#22] -Results [3]: [item_sk#10, d_date#6, sum(sumws#15)#22 AS cume_sales#23] -(30) CometColumnarExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(27) CometExchange +Input [3]: [item_sk#10, d_date#6, cume_sales#20] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(31) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#23] -Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(28) CometSort +Input [3]: [item_sk#10, d_date#6, cume_sales#20] +Arguments: [item_sk#10, d_date#6, cume_sales#20], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] +PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(33) CometFilter -Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Condition : isnotnull(ss_item_sk#24) - -(34) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#28, d_date#29] - -(35) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] -Right output [2]: [d_date_sk#28, d_date#29] -Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight - -(36) CometProject -Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] -Arguments: [ss_item_sk#24, ss_sales_price#25, d_date#29], [ss_item_sk#24, ss_sales_price#25, d_date#29] - -(37) CometHashAggregate -Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] - -(38) CometExchange -Input [3]: [ss_item_sk#24, d_date#29, sum#30] -Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(39) CometColumnarToRow [codegen id : 8] -Input [3]: [ss_item_sk#24, d_date#29, sum#30] - -(40) HashAggregate [codegen id : 8] -Input [3]: [ss_item_sk#24, d_date#29, sum#30] -Keys [2]: [ss_item_sk#24, d_date#29] -Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] -Results [4]: [ss_item_sk#24 AS item_sk#32, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS sumss#33, ss_item_sk#24] - -(41) CometColumnarExchange -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] - -(42) CometSort -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -Arguments: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] - -(43) CometColumnarToRow [codegen id : 9] -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +(30) CometFilter +Input [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] +Condition : isnotnull(ss_item_sk#21) + +(31) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#25, d_date#26] + +(32) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] +Right output [2]: [d_date_sk#25, d_date#26] +Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight + +(33) CometProject +Input [5]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23, d_date_sk#25, d_date#26] +Arguments: [ss_item_sk#21, ss_sales_price#22, d_date#26], [ss_item_sk#21, ss_sales_price#22, d_date#26] + +(34) CometHashAggregate +Input [3]: [ss_item_sk#21, ss_sales_price#22, d_date#26] +Keys [2]: [ss_item_sk#21, d_date#26] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#22))] + +(35) CometExchange +Input [3]: [ss_item_sk#21, d_date#26, sum#27] +Arguments: hashpartitioning(ss_item_sk#21, d_date#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(36) CometColumnarToRow [codegen id : 3] +Input [3]: [ss_item_sk#21, d_date#26, sum#27] + +(37) HashAggregate [codegen id : 3] +Input [3]: [ss_item_sk#21, d_date#26, sum#27] +Keys [2]: [ss_item_sk#21, d_date#26] +Functions [1]: [sum(UnscaledValue(ss_sales_price#22))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#22))#28] +Results [4]: [ss_item_sk#21 AS item_sk#29, d_date#26, MakeDecimal(sum(UnscaledValue(ss_sales_price#22))#28,17,2) AS sumss#30, ss_item_sk#21] + +(38) CometColumnarExchange +Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] +Arguments: hashpartitioning(ss_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] + +(39) CometSort +Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] +Arguments: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21], [ss_item_sk#21 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST] + +(40) CometWindowExec +Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] +Arguments: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21, rk#31], [row_number() windowspecdefinition(ss_item_sk#21, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#31], [ss_item_sk#21], [d_date#26 ASC NULLS FIRST] + +(41) CometProject +Input [5]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21, rk#31] +Arguments: [item_sk#29, d_date#26, sumss#30, rk#31], [item_sk#29, d_date#26, sumss#30, rk#31] + +(42) ReusedExchange [Reuses operator id: 38] +Output [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] + +(43) CometSort +Input [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] +Arguments: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35], [ss_item_sk#35 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] + +(44) CometWindowExec +Input [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] +Arguments: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35, rk#36], [row_number() windowspecdefinition(ss_item_sk#35, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [ss_item_sk#35], [d_date#33 ASC NULLS FIRST] + +(45) CometProject +Input [5]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35, rk#36] +Arguments: [item_sk#32, sumss#34, rk#36], [item_sk#32, sumss#34, rk#36] + +(46) CometBroadcastExchange +Input [3]: [item_sk#32, sumss#34, rk#36] +Arguments: [item_sk#32, sumss#34, rk#36] + +(47) CometBroadcastHashJoin +Left output [4]: [item_sk#29, d_date#26, sumss#30, rk#31] +Right output [3]: [item_sk#32, sumss#34, rk#36] +Arguments: [item_sk#29], [item_sk#32], Inner, (rk#31 >= rk#36), BuildRight + +(48) CometProject +Input [7]: [item_sk#29, d_date#26, sumss#30, rk#31, item_sk#32, sumss#34, rk#36] +Arguments: [item_sk#29, d_date#26, sumss#30, sumss#34], [item_sk#29, d_date#26, sumss#30, sumss#34] + +(49) CometHashAggregate +Input [4]: [item_sk#29, d_date#26, sumss#30, sumss#34] +Keys [3]: [item_sk#29, d_date#26, sumss#30] +Functions [1]: [partial_sum(sumss#34)] + +(50) CometExchange +Input [5]: [item_sk#29, d_date#26, sumss#30, sum#37, isEmpty#38] +Arguments: hashpartitioning(item_sk#29, d_date#26, sumss#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(51) CometHashAggregate +Input [5]: [item_sk#29, d_date#26, sumss#30, sum#37, isEmpty#38] +Keys [3]: [item_sk#29, d_date#26, sumss#30] +Functions [1]: [sum(sumss#34)] + +(52) CometExchange +Input [3]: [item_sk#29, d_date#26, cume_sales#39] +Arguments: hashpartitioning(item_sk#29, d_date#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(53) CometSort +Input [3]: [item_sk#29, d_date#26, cume_sales#39] +Arguments: [item_sk#29, d_date#26, cume_sales#39], [item_sk#29 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST] + +(54) CometSortMergeJoin +Left output [3]: [item_sk#10, d_date#6, cume_sales#20] +Right output [3]: [item_sk#29, d_date#26, cume_sales#39] +Arguments: [item_sk#10, d_date#6], [item_sk#29, d_date#26], FullOuter + +(55) CometFilter +Input [6]: [item_sk#10, d_date#6, cume_sales#20, item_sk#29, d_date#26, cume_sales#39] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#29 END) + +(56) CometProject +Input [6]: [item_sk#10, d_date#6, cume_sales#20, item_sk#29, d_date#26, cume_sales#39] +Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#29 END AS item_sk#40, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#26 END AS d_date#41, cume_sales#20 AS web_sales#42, cume_sales#39 AS store_sales#43] + +(57) CometExchange +Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] +Arguments: hashpartitioning(item_sk#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] + +(58) CometSort +Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] +Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43], [item_sk#40 ASC NULLS FIRST, d_date#41 ASC NULLS FIRST] + +(59) CometWindowExec +Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] +Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44], [row_number() windowspecdefinition(item_sk#40, d_date#41 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#44], [item_sk#40], [d_date#41 ASC NULLS FIRST] + +(60) ReusedExchange [Reuses operator id: 57] +Output [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] + +(61) CometSort +Input [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] +Arguments: [item_sk#45, d_date#46, web_sales#47, store_sales#48], [item_sk#45 ASC NULLS FIRST, d_date#46 ASC NULLS FIRST] + +(62) CometWindowExec +Input [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] +Arguments: [item_sk#45, d_date#46, web_sales#47, store_sales#48, rk#49], [row_number() windowspecdefinition(item_sk#45, d_date#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#45], [d_date#46 ASC NULLS FIRST] -(44) Window -Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] +(63) CometProject +Input [5]: [item_sk#45, d_date#46, web_sales#47, store_sales#48, rk#49] +Arguments: [item_sk#45, web_sales#47, store_sales#48, rk#49], [item_sk#45, web_sales#47, store_sales#48, rk#49] -(45) Project [codegen id : 13] -Output [4]: [item_sk#32, d_date#29, sumss#33, rk#34] -Input [5]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24, rk#34] +(64) CometBroadcastExchange +Input [4]: [item_sk#45, web_sales#47, store_sales#48, rk#49] +Arguments: [item_sk#45, web_sales#47, store_sales#48, rk#49] -(46) ReusedExchange [Reuses operator id: 41] -Output [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +(65) CometBroadcastHashJoin +Left output [5]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44] +Right output [4]: [item_sk#45, web_sales#47, store_sales#48, rk#49] +Arguments: [item_sk#40], [item_sk#45], Inner, (rk#44 >= rk#49), BuildRight -(47) CometSort -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -Arguments: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38], [ss_item_sk#38 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] +(66) CometProject +Input [9]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44, item_sk#45, web_sales#47, store_sales#48, rk#49] +Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48], [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48] -(48) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] - -(49) Window -Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -Arguments: [row_number() windowspecdefinition(ss_item_sk#38, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#39], [ss_item_sk#38], [d_date#36 ASC NULLS FIRST] - -(50) Project [codegen id : 12] -Output [3]: [item_sk#35, sumss#37, rk#39] -Input [5]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38, rk#39] - -(51) BroadcastExchange -Input [3]: [item_sk#35, sumss#37, rk#39] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(52) BroadcastHashJoin [codegen id : 13] -Left keys [1]: [item_sk#32] -Right keys [1]: [item_sk#35] -Join type: Inner -Join condition: (rk#34 >= rk#39) - -(53) Project [codegen id : 13] -Output [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] -Input [7]: [item_sk#32, d_date#29, sumss#33, rk#34, item_sk#35, sumss#37, rk#39] - -(54) HashAggregate [codegen id : 13] -Input [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] -Keys [3]: [item_sk#32, d_date#29, sumss#33] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#40, isEmpty#41] -Results [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] - -(55) CometColumnarExchange -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] -Arguments: hashpartitioning(item_sk#32, d_date#29, sumss#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(56) CometColumnarToRow [codegen id : 14] -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] - -(57) HashAggregate [codegen id : 14] -Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] -Keys [3]: [item_sk#32, d_date#29, sumss#33] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#44] -Results [3]: [item_sk#32, d_date#29, sum(sumss#37)#44 AS cume_sales#45] - -(58) CometColumnarExchange -Input [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: hashpartitioning(item_sk#32, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(59) CometSort -Input [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#32, d_date#29, cume_sales#45], [item_sk#32 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] - -(60) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#23] -Right output [3]: [item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#10, d_date#6], [item_sk#32, d_date#29], FullOuter - -(61) CometFilter -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END) - -(62) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END AS item_sk#46, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#47, cume_sales#23 AS web_sales#48, cume_sales#45 AS store_sales#49] - -(63) CometExchange -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(64) CometSort -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] - -(65) CometColumnarToRow [codegen id : 15] -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] - -(66) Window -Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 ASC NULLS FIRST] - -(67) ReusedExchange [Reuses operator id: 63] -Output [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] - -(68) CometSort -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] -Arguments: [item_sk#51, d_date#52, web_sales#53, store_sales#54], [item_sk#51 ASC NULLS FIRST, d_date#52 ASC NULLS FIRST] - -(69) CometColumnarToRow [codegen id : 30] -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] - -(70) Window -Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] -Arguments: [row_number() windowspecdefinition(item_sk#51, d_date#52 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#55], [item_sk#51], [d_date#52 ASC NULLS FIRST] - -(71) Project [codegen id : 31] -Output [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] -Input [5]: [item_sk#51, d_date#52, web_sales#53, store_sales#54, rk#55] - -(72) BroadcastExchange -Input [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(73) BroadcastHashJoin [codegen id : 32] -Left keys [1]: [item_sk#46] -Right keys [1]: [item_sk#51] -Join type: Inner -Join condition: (rk#50 >= rk#55) - -(74) Project [codegen id : 32] -Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#51, web_sales#53, store_sales#54, rk#55] - -(75) HashAggregate [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] -Aggregate Attributes [2]: [max#56, max#57] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] - -(76) HashAggregate [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] -Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] -Functions [2]: [max(web_sales#53), max(store_sales#54)] -Aggregate Attributes [2]: [max(web_sales#53)#60, max(store_sales#54)#61] -Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#60 AS web_cumulative#62, max(store_sales#54)#61 AS store_cumulative#63] - -(77) Filter [codegen id : 32] -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] -Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) - -(78) TakeOrderedAndProject -Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] -Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +(67) CometHashAggregate +Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48] +Keys [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] +Functions [2]: [partial_max(web_sales#47), partial_max(store_sales#48)] + +(68) CometHashAggregate +Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, max#50, max#51] +Keys [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] +Functions [2]: [max(web_sales#47), max(store_sales#48)] + +(69) CometFilter +Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] +Condition : ((isnotnull(web_cumulative#52) AND isnotnull(store_cumulative#53)) AND (web_cumulative#52 > store_cumulative#53)) + +(70) CometTakeOrderedAndProject +Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#40 ASC NULLS FIRST,d_date#41 ASC NULLS FIRST], output=[item_sk#40,d_date#41,web_sales#42,store_sales#43,web_cumulative#52,store_cumulative#53]), [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53], 100, 0, [item_sk#40 ASC NULLS FIRST, d_date#41 ASC NULLS FIRST], [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] + +(71) CometColumnarToRow [codegen id : 9] +Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter +(73) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(81) CometProject +(74) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(82) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(83) BroadcastExchange +(76) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt index f859e0bf3e..6e3fd36930 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt @@ -1,224 +1,211 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometWindowExec + : : : +- CometSort + : : : +- CometColumnarExchange + : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : : +- CometColumnarToRow + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 130 out of 196 eligible operators (66%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 176 out of 196 eligible operators (89%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt index c88883dbad..51d1e38ec6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt @@ -1,122 +1,86 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (32) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (15) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (7) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (6) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] +WholeStageCodegen (9) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometHashAggregate [max,max] [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative,max(web_sales),max(store_sales)] + CometHashAggregate [web_sales,store_sales] [item_sk,d_date,web_sales,store_sales,max,max] + CometProject [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + CometBroadcastHashJoin [item_sk,d_date,web_sales,store_sales,rk,item_sk,web_sales,store_sales,rk] + CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #2 + CometHashAggregate [sumws,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumws)] + CometExchange [item_sk,d_date,sumws] #3 + CometHashAggregate [sumws] [item_sk,d_date,sumws,sum,isEmpty] + CometProject [item_sk,d_date,sumws,sumws] + CometBroadcastHashJoin [item_sk,d_date,sumws,rk,item_sk,sumws,rk] + CometProject [item_sk,d_date,sumws,rk] + CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + CometColumnarToRow InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow - InputAdapter - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (5) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (14) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (13) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] + CometExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [item_sk,sumws,rk] #8 + CometProject [item_sk,sumws,rk] + CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #9 + CometHashAggregate [sumss,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumss)] + CometExchange [item_sk,d_date,sumss] #10 + CometHashAggregate [sumss] [item_sk,d_date,sumss,sum,isEmpty] + CometProject [item_sk,d_date,sumss,sumss] + CometBroadcastHashJoin [item_sk,d_date,sumss,rk,item_sk,sumss,rk] + CometProject [item_sk,d_date,sumss,rk] + CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #11 + WholeStageCodegen (3) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + CometColumnarToRow InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #11 - WholeStageCodegen (8) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (12) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (31) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (30) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 + CometBroadcastExchange [item_sk,sumss,rk] #13 + CometProject [item_sk,sumss,rk] + CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + CometBroadcastExchange [item_sk,web_sales,store_sales,rk] #14 + CometProject [item_sk,web_sales,store_sales,rk] + CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt index e5b34056ab..0181f2dc7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt @@ -1,54 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (29) + : : +- CometFilter (28) + : : +- CometWindowExec (27) + : : +- CometFilter (26) + : : +- CometWindowExec (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- CometBroadcastExchange (37) + : +- CometProject (36) + : +- CometWindowExec (35) + : +- CometSort (34) + : +- CometColumnarExchange (33) + : +- * HashAggregate (32) + : +- * CometColumnarToRow (31) + : +- ReusedExchange (30) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -166,136 +160,112 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 2] +(25) CometWindowExec Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(27) Filter [codegen id : 3] +(26) CometFilter Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(28) Window +(27) CometWindowExec Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(29) Filter [codegen id : 10] +(28) CometFilter Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(30) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +(29) CometProject Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(31) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(32) CometColumnarToRow [codegen id : 4] +(31) CometColumnarToRow [codegen id : 2] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(33) HashAggregate [codegen id : 4] +(32) HashAggregate [codegen id : 2] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] -(34) CometColumnarExchange +(33) CometColumnarExchange Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(35) CometSort +(34) CometSort Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] - -(37) Window +(35) CometWindowExec Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30], [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(36) CometProject Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] +Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30], [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(39) BroadcastExchange +(37) CometBroadcastExchange Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +Right output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)], Inner, BuildRight -(41) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] +(39) CometProject Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -(42) ReusedExchange [Reuses operator id: 34] -Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] - -(43) CometSort -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 8] -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] - -(45) Window -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(46) Project [codegen id : 9] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] +(40) ReusedExchange [Reuses operator id: 37] +Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -(47) BroadcastExchange -Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] +Right output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#31, i_brand#32, cc_name#33, (rn#35 - 1)], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 1)] -Join type: Inner -Join condition: None +(42) CometProject +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +Arguments: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#36, sum_sales#34 AS nsum#37] -(49) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(43) CometTakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#36,nsum#37]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] -(50) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(44) CometColumnarToRow [codegen id : 4] +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(53) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) BroadcastExchange +(48) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt index 8c7324af76..8d50c97859 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt @@ -1,80 +1,78 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] @@ -105,4 +103,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt index 29a73f88ac..8f7680b9f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt @@ -1,77 +1,57 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (10) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (3) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (2) +WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,cc_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] CometColumnarToRow InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt index e5b34056ab..0181f2dc7f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt @@ -1,54 +1,48 @@ == Physical Plan == -TakeOrderedAndProject (50) -+- * Project (49) - +- * BroadcastHashJoin Inner BuildRight (48) - :- * Project (41) - : +- * BroadcastHashJoin Inner BuildRight (40) - : :- * Project (30) - : : +- * Filter (29) - : : +- Window (28) - : : +- * Filter (27) - : : +- Window (26) - : : +- * CometColumnarToRow (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (39) - : +- * Project (38) - : +- Window (37) - : +- * CometColumnarToRow (36) - : +- CometSort (35) - : +- CometColumnarExchange (34) - : +- * HashAggregate (33) - : +- * CometColumnarToRow (32) - : +- ReusedExchange (31) - +- BroadcastExchange (47) - +- * Project (46) - +- Window (45) - +- * CometColumnarToRow (44) - +- CometSort (43) - +- ReusedExchange (42) +* CometColumnarToRow (44) ++- CometTakeOrderedAndProject (43) + +- CometProject (42) + +- CometBroadcastHashJoin (41) + :- CometProject (39) + : +- CometBroadcastHashJoin (38) + : :- CometProject (29) + : : +- CometFilter (28) + : : +- CometWindowExec (27) + : : +- CometFilter (26) + : : +- CometWindowExec (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- CometBroadcastExchange (37) + : +- CometProject (36) + : +- CometWindowExec (35) + : +- CometSort (34) + : +- CometColumnarExchange (33) + : +- * HashAggregate (32) + : +- * CometColumnarToRow (31) + : +- ReusedExchange (30) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -166,136 +160,112 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 2] +(25) CometWindowExec Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(27) Filter [codegen id : 3] +(26) CometFilter Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(28) Window +(27) CometWindowExec Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(29) Filter [codegen id : 10] +(28) CometFilter Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(30) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +(29) CometProject Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(31) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(32) CometColumnarToRow [codegen id : 4] +(31) CometColumnarToRow [codegen id : 2] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(33) HashAggregate [codegen id : 4] +(32) HashAggregate [codegen id : 2] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] -(34) CometColumnarExchange +(33) CometColumnarExchange Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(35) CometSort +(34) CometSort Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(36) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] - -(37) Window +(35) CometWindowExec Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30], [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(38) Project [codegen id : 6] -Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +(36) CometProject Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] +Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30], [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(39) BroadcastExchange +(37) CometBroadcastExchange Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(40) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)] -Join type: Inner -Join condition: None +(38) CometBroadcastHashJoin +Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +Right output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)], Inner, BuildRight -(41) Project [codegen id : 10] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] +(39) CometProject Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -(42) ReusedExchange [Reuses operator id: 34] -Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] - -(43) CometSort -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) CometColumnarToRow [codegen id : 8] -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] - -(45) Window -Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(46) Project [codegen id : 9] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] +(40) ReusedExchange [Reuses operator id: 37] +Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -(47) BroadcastExchange -Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] +(41) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] +Right output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#31, i_brand#32, cc_name#33, (rn#35 - 1)], Inner, BuildRight -(48) BroadcastHashJoin [codegen id : 10] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] -Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 1)] -Join type: Inner -Join condition: None +(42) CometProject +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +Arguments: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#36, sum_sales#34 AS nsum#37] -(49) Project [codegen id : 10] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +(43) CometTakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#36,nsum#37]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] -(50) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(44) CometColumnarToRow [codegen id : 4] +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (54) -+- * CometColumnarToRow (53) - +- CometFilter (52) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometFilter (46) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) -(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(52) CometFilter +(46) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(53) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(54) BroadcastExchange +(48) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt index 8c7324af76..8d50c97859 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt @@ -1,80 +1,78 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] @@ -105,4 +103,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt index 29a73f88ac..8f7680b9f5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt @@ -1,77 +1,57 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (10) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (3) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (2) +WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,cc_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (2) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] CometColumnarToRow InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (4) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (9) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (8) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt index ae9541ff99..ac091ccfdd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (66) -+- * Project (65) - +- Window (64) - +- * CometColumnarToRow (63) +* CometColumnarToRow (66) ++- CometTakeOrderedAndProject (65) + +- CometProject (64) + +- CometWindowExec (63) +- CometSort (62) +- CometExchange (61) +- CometHashAggregate (60) @@ -358,20 +358,20 @@ Arguments: hashpartitioning(lochierarchy#31, _w0#63, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63], [lochierarchy#31 ASC NULLS FIRST, _w0#63 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST] -(63) CometColumnarToRow [codegen id : 19] +(63) CometWindowExec Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] +Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64], [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#63, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#64], [lochierarchy#31, _w0#63], [total_sum#26 DESC NULLS LAST] -(64) Window -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#63, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#64], [lochierarchy#31, _w0#63], [total_sum#26 DESC NULLS LAST] - -(65) Project [codegen id : 20] -Output [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +(64) CometProject Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64] +Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] + +(65) CometTakeOrderedAndProject +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#31 DESC NULLS LAST,CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST,rank_within_parent#64 ASC NULLS FIRST], output=[total_sum#26,s_state#27,s_county#28,lochierarchy#31,rank_within_parent#64]), [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64], 100, 0, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#64 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] -(66) TakeOrderedAndProject +(66) CometColumnarToRow [codegen id : 19] Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] -Arguments: 100, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#64 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt index df5794808f..d73b57ca68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -173,4 +173,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 96 out of 156 eligible operators (61%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 99 out of 156 eligible operators (63%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt index 21c6dd4eb1..3f4387c7ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt @@ -1,101 +1,99 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 +WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,rank_within_parent] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + WholeStageCodegen (18) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + CometColumnarToRow + InputAdapter + CometExchange [s_state] #8 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #9 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (11) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #10 + WholeStageCodegen (10) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #11 + WholeStageCodegen (16) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt index ae9541ff99..ac091ccfdd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (66) -+- * Project (65) - +- Window (64) - +- * CometColumnarToRow (63) +* CometColumnarToRow (66) ++- CometTakeOrderedAndProject (65) + +- CometProject (64) + +- CometWindowExec (63) +- CometSort (62) +- CometExchange (61) +- CometHashAggregate (60) @@ -358,20 +358,20 @@ Arguments: hashpartitioning(lochierarchy#31, _w0#63, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63], [lochierarchy#31 ASC NULLS FIRST, _w0#63 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST] -(63) CometColumnarToRow [codegen id : 19] +(63) CometWindowExec Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] +Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64], [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#63, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#64], [lochierarchy#31, _w0#63], [total_sum#26 DESC NULLS LAST] -(64) Window -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] -Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#63, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#64], [lochierarchy#31, _w0#63], [total_sum#26 DESC NULLS LAST] - -(65) Project [codegen id : 20] -Output [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +(64) CometProject Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64] +Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] + +(65) CometTakeOrderedAndProject +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#31 DESC NULLS LAST,CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST,rank_within_parent#64 ASC NULLS FIRST], output=[total_sum#26,s_state#27,s_county#28,lochierarchy#31,rank_within_parent#64]), [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64], 100, 0, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#64 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] -(66) TakeOrderedAndProject +(66) CometColumnarToRow [codegen id : 19] Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] -Arguments: 100, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#64 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt index df5794808f..d73b57ca68 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -173,4 +173,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 96 out of 156 eligible operators (61%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 99 out of 156 eligible operators (63%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt index 21c6dd4eb1..3f4387c7ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt @@ -1,101 +1,99 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 +WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,rank_within_parent] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + WholeStageCodegen (18) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + CometColumnarToRow + InputAdapter + CometExchange [s_state] #8 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #9 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (11) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #10 + WholeStageCodegen (10) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #11 + WholeStageCodegen (16) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt index 07622631b8..b8e5e4175c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- Window (40) - +- * CometColumnarToRow (39) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometWindowExec (39) +- CometSort (38) +- CometExchange (37) +- CometHashAggregate (36) @@ -224,20 +224,20 @@ Arguments: hashpartitioning(lochierarchy#19, _w0#51, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51], [lochierarchy#19 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] -(39) CometColumnarToRow [codegen id : 7] +(39) CometWindowExec Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] +Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52], [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] -(40) Window -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] - -(41) Project [codegen id : 8] -Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +(40) CometProject Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52] +Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] + +(41) CometTakeOrderedAndProject +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#19 DESC NULLS LAST,CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST,rank_within_parent#52 ASC NULLS FIRST], output=[total_sum#14,i_category#15,i_class#16,lochierarchy#19,rank_within_parent#52]), [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52], 100, 0, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -(42) TakeOrderedAndProject +(42) CometColumnarToRow [codegen id : 7] Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt index 26bf732bde..9dea0e7de6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -89,4 +89,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 63 out of 81 eligible operators (77%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 81 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt index d9db895fba..d1903ae6b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt @@ -1,66 +1,64 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (8) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (6) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (2) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (5) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (4) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 +WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,rank_within_parent] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + WholeStageCodegen (6) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (1) + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class] #3 + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (3) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #7 + WholeStageCodegen (2) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 + WholeStageCodegen (5) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #8 + WholeStageCodegen (4) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt index 07622631b8..b8e5e4175c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (42) -+- * Project (41) - +- Window (40) - +- * CometColumnarToRow (39) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometWindowExec (39) +- CometSort (38) +- CometExchange (37) +- CometHashAggregate (36) @@ -224,20 +224,20 @@ Arguments: hashpartitioning(lochierarchy#19, _w0#51, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51], [lochierarchy#19 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] -(39) CometColumnarToRow [codegen id : 7] +(39) CometWindowExec Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] +Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52], [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] -(40) Window -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] - -(41) Project [codegen id : 8] -Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +(40) CometProject Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52] +Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] + +(41) CometTakeOrderedAndProject +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#19 DESC NULLS LAST,CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST,rank_within_parent#52 ASC NULLS FIRST], output=[total_sum#14,i_category#15,i_class#16,lochierarchy#19,rank_within_parent#52]), [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52], 100, 0, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -(42) TakeOrderedAndProject +(42) CometColumnarToRow [codegen id : 7] Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt index 26bf732bde..9dea0e7de6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -89,4 +89,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 63 out of 81 eligible operators (77%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 81 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt index d9db895fba..d1903ae6b4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt @@ -1,66 +1,64 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (8) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (6) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (2) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (5) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (4) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 +WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,rank_within_parent] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + WholeStageCodegen (6) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (1) + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class] #3 + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (3) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #7 + WholeStageCodegen (2) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 + WholeStageCodegen (5) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #8 + WholeStageCodegen (4) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt index 4b24124687..92a6fc5a00 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt @@ -1,30 +1,29 @@ == Physical Plan == -* CometColumnarToRow (26) -+- CometSort (25) - +- CometColumnarExchange (24) - +- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (25) ++- CometSort (24) + +- CometExchange (23) + +- CometProject (22) + +- CometWindowExec (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -122,57 +121,54 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +(22) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -(24) CometColumnarExchange +(23) CometExchange Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(25) CometSort +(24) CometSort Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 4] +(25) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(29) CometProject +(28) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(31) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt index e6a60f7a0a..5ee2c01b35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt @@ -1,34 +1,33 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- Window - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS _w0#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS _w0#9)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt index 1a79190465..8067c884b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt @@ -1,42 +1,37 @@ -WholeStageCodegen (4) +WholeStageCodegen (2) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (3) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt index 4b24124687..92a6fc5a00 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt @@ -1,30 +1,29 @@ == Physical Plan == -* CometColumnarToRow (26) -+- CometSort (25) - +- CometColumnarExchange (24) - +- * Project (23) - +- Window (22) - +- * CometColumnarToRow (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (25) ++- CometSort (24) + +- CometExchange (23) + +- CometProject (22) + +- CometWindowExec (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -122,57 +121,54 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 2] +(21) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] - -(23) Project [codegen id : 3] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +(22) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -(24) CometColumnarExchange +(23) CometExchange Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(25) CometSort +(24) CometSort Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(26) CometColumnarToRow [codegen id : 4] +(25) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(29) CometProject +(28) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(31) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt index e6a60f7a0a..5ee2c01b35 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt @@ -1,34 +1,33 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- Window - +- CometColumnarToRow - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS _w0#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS _w0#9)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt index 1a79190465..8067c884b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt @@ -1,42 +1,37 @@ -WholeStageCodegen (4) +WholeStageCodegen (2) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (3) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt index 410aecff1b..db7ab3e60c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) +* CometColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometProject (21) + +- CometWindowExec (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +(21) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] + +(22) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -(23) TakeOrderedAndProject +(23) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt index 2708ea9eae..7b9553ba1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt index 58ac81ba0d..63d522180c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt @@ -1,35 +1,33 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index 410aecff1b..db7ab3e60c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) +* CometColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometProject (21) + +- CometWindowExec (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +(21) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] + +(22) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -(23) TakeOrderedAndProject +(23) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt index 2708ea9eae..7b9553ba1c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 58ac81ba0d..63d522180c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -1,35 +1,33 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt index 322bbe63b3..e4b07ef2a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) +* CometColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometProject (21) + +- CometWindowExec (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +(21) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] + +(22) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -(23) TakeOrderedAndProject +(23) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt index b08f4f6d0e..3bd3bd46f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt index cf18e68a3d..173aea3be4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt @@ -1,35 +1,33 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index 322bbe63b3..e4b07ef2a8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (23) -+- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) +* CometColumnarToRow (23) ++- CometTakeOrderedAndProject (22) + +- CometProject (21) + +- CometWindowExec (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +(21) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] + +(22) CometTakeOrderedAndProject +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -(23) TakeOrderedAndProject +(23) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt index b08f4f6d0e..3bd3bd46f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index cf18e68a3d..173aea3be4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -1,35 +1,33 @@ -TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt index 0f966424f8..c0a8c11952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * CometColumnarToRow (40) +* CometColumnarToRow (43) ++- CometTakeOrderedAndProject (42) + +- CometProject (41) + +- CometWindowExec (40) +- CometSort (39) +- CometExchange (38) +- CometHashAggregate (37) @@ -225,20 +225,20 @@ Arguments: hashpartitioning(lochierarchy#41, _w0#53, 5), ENSURE_REQUIREMENTS, Co Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53], [lochierarchy#41 ASC NULLS FIRST, _w0#53 ASC NULLS FIRST, gross_margin#36 ASC NULLS FIRST] -(40) CometColumnarToRow [codegen id : 1] +(40) CometWindowExec Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] +Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54], [rank(gross_margin#36) windowspecdefinition(lochierarchy#41, _w0#53, gross_margin#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#54], [lochierarchy#41, _w0#53], [gross_margin#36 ASC NULLS FIRST] -(41) Window -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] -Arguments: [rank(gross_margin#36) windowspecdefinition(lochierarchy#41, _w0#53, gross_margin#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#54], [lochierarchy#41, _w0#53], [gross_margin#36 ASC NULLS FIRST] - -(42) Project [codegen id : 2] -Output [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +(41) CometProject Input [6]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54] +Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] + +(42) CometTakeOrderedAndProject +Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#41 DESC NULLS LAST,CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST,rank_within_parent#54 ASC NULLS FIRST], output=[gross_margin#36,i_category#37,i_class#38,lochierarchy#41,rank_within_parent#54]), [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54], 100, 0, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST, rank_within_parent#54 ASC NULLS FIRST], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] -(43) TakeOrderedAndProject +(43) CometColumnarToRow [codegen id : 1] Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] -Arguments: 100, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST, rank_within_parent#54 ASC NULLS FIRST], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt index f0fa5a92af..cd5c2362b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +102,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt index e91b278c14..70d1ad26f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt @@ -1,55 +1,53 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,rank_within_parent] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange [i_category] #8 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange #9 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt index 0f966424f8..c0a8c11952 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (43) -+- * Project (42) - +- Window (41) - +- * CometColumnarToRow (40) +* CometColumnarToRow (43) ++- CometTakeOrderedAndProject (42) + +- CometProject (41) + +- CometWindowExec (40) +- CometSort (39) +- CometExchange (38) +- CometHashAggregate (37) @@ -225,20 +225,20 @@ Arguments: hashpartitioning(lochierarchy#41, _w0#53, 5), ENSURE_REQUIREMENTS, Co Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53], [lochierarchy#41 ASC NULLS FIRST, _w0#53 ASC NULLS FIRST, gross_margin#36 ASC NULLS FIRST] -(40) CometColumnarToRow [codegen id : 1] +(40) CometWindowExec Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] +Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54], [rank(gross_margin#36) windowspecdefinition(lochierarchy#41, _w0#53, gross_margin#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#54], [lochierarchy#41, _w0#53], [gross_margin#36 ASC NULLS FIRST] -(41) Window -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] -Arguments: [rank(gross_margin#36) windowspecdefinition(lochierarchy#41, _w0#53, gross_margin#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#54], [lochierarchy#41, _w0#53], [gross_margin#36 ASC NULLS FIRST] - -(42) Project [codegen id : 2] -Output [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +(41) CometProject Input [6]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54] +Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] + +(42) CometTakeOrderedAndProject +Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#41 DESC NULLS LAST,CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST,rank_within_parent#54 ASC NULLS FIRST], output=[gross_margin#36,i_category#37,i_class#38,lochierarchy#41,rank_within_parent#54]), [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54], 100, 0, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST, rank_within_parent#54 ASC NULLS FIRST], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] -(43) TakeOrderedAndProject +(43) CometColumnarToRow [codegen id : 1] Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] -Arguments: 100, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST, rank_within_parent#54 ASC NULLS FIRST], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt index f0fa5a92af..cd5c2362b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +102,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index e91b278c14..70d1ad26f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -1,55 +1,53 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] - WholeStageCodegen (2) - Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [gross_margin,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,rank_within_parent] + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange [i_category] #8 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange #9 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt index 7b1f860f32..53bc60ca3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt @@ -1,52 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (28) + : : +- CometFilter (27) + : : +- CometWindowExec (26) + : : +- CometFilter (25) + : : +- CometWindowExec (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometWindowExec (33) + : +- CometSort (32) + : +- CometExchange (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + +- ReusedExchange (38) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -159,131 +153,107 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] +(24) CometWindowExec Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] +(25) CometFilter Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window +(26) CometWindowExec Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(28) Filter [codegen id : 7] +(27) CometFilter Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +(28) CometProject Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(30) ReusedExchange [Reuses operator id: 20] +(29) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -(31) CometHashAggregate +(30) CometHashAggregate Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] -(32) CometExchange +(31) CometExchange Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(33) CometSort +(32) CometSort Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window +(33) CometWindowExec Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28], [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] +(34) CometProject Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] +Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28], [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -(37) BroadcastExchange +(35) CometBroadcastExchange Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +Right output [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)], Inner, BuildRight -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +(37) CometProject Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] +(38) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +Right output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#37 - 1)], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None +(40) CometProject +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +Arguments: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#38, sum_sales#36 AS nsum#39] -(47) Project [codegen id : 7] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +(41) CometTakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#38,nsum#39]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -(48) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +(42) CometColumnarToRow [codegen id : 1] +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(44) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(46) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt index 1bbdb7273f..8c361a8340 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt @@ -1,78 +1,76 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt index a62c33ecc1..c8c2cc4f95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt @@ -1,71 +1,51 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (7) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index 7b1f860f32..53bc60ca3a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -1,52 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (28) + : : +- CometFilter (27) + : : +- CometWindowExec (26) + : : +- CometFilter (25) + : : +- CometWindowExec (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometWindowExec (33) + : +- CometSort (32) + : +- CometExchange (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + +- ReusedExchange (38) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -159,131 +153,107 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] +(24) CometWindowExec Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) Window -Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] +(25) CometFilter Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window +(26) CometWindowExec Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(28) Filter [codegen id : 7] +(27) CometFilter Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +(28) CometProject Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(30) ReusedExchange [Reuses operator id: 20] +(29) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -(31) CometHashAggregate +(30) CometHashAggregate Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] -(32) CometExchange +(31) CometExchange Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(33) CometSort +(32) CometSort Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(34) CometColumnarToRow [codegen id : 3] -Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] - -(35) Window +(33) CometWindowExec Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28], [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(36) Project [codegen id : 4] -Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] +(34) CometProject Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] +Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28], [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -(37) BroadcastExchange +(35) CometBroadcastExchange Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] +Right output [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)], Inner, BuildRight -(39) Project [codegen id : 7] -Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +(37) CometProject Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -(40) ReusedExchange [Reuses operator id: 32] -Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(41) CometSort -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] - -(43) Window -Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] -Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] -Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] +(38) ReusedExchange [Reuses operator id: 35] +Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -(45) BroadcastExchange -Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] +Right output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#37 - 1)], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 7] -Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] -Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] -Join type: Inner -Join condition: None +(40) CometProject +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +Arguments: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#38, sum_sales#36 AS nsum#39] -(47) Project [codegen id : 7] -Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +(41) CometTakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#38,nsum#39]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -(48) TakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] -Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +(42) CometColumnarToRow [codegen id : 1] +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(44) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(46) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt index 1bbdb7273f..8c361a8340 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt @@ -1,78 +1,76 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index a62c33ecc1..c8c2cc4f95 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -1,71 +1,51 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] - WholeStageCodegen (7) - Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] - Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt index 74702d596d..2c993b48e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt @@ -1,82 +1,79 @@ == Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) +* CometColumnarToRow (75) ++- CometTakeOrderedAndProject (74) + +- CometHashAggregate (73) + +- CometExchange (72) + +- CometHashAggregate (71) + +- CometUnion (70) + :- CometProject (25) + : +- CometFilter (24) + : +- CometWindowExec (23) + : +- CometSort (22) + : +- CometWindowExec (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- CometProject (47) + : +- CometFilter (46) + : +- CometWindowExec (45) + : +- CometSort (44) + : +- CometWindowExec (43) + : +- CometSort (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometExchange (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometBroadcastExchange (29) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (35) + +- CometProject (69) + +- CometFilter (68) + +- CometWindowExec (67) + +- CometSort (66) + +- CometWindowExec (65) + +- CometSort (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometProject (56) + : +- CometBroadcastHashJoin (55) + : :- CometBroadcastExchange (51) + : : +- CometProject (50) + : : +- CometFilter (49) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) + +- ReusedExchange (57) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -173,30 +170,27 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 1] +(21) CometWindowExec Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] +(22) CometSort Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] -(24) Window +(23) CometWindowExec Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(25) Filter [codegen id : 3] +(24) CometFilter Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(25) CometProject Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -204,100 +198,97 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(29) CometProject +(28) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometBroadcastExchange +(29) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(33) CometProject +(32) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(34) CometBroadcastHashJoin +(33) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(35) CometProject +(34) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(36) ReusedExchange [Reuses operator id: 13] +(35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(37) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(38) CometProject +(37) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(39) CometHashAggregate +(38) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(40) CometExchange +(39) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(41) CometHashAggregate +(40) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(42) CometExchange +(41) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort +(42) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window +(43) CometWindowExec Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(46) Sort [codegen id : 5] +(44) CometSort Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] -(47) Window +(45) CometWindowExec Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(48) Filter [codegen id : 6] +(46) CometFilter Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +(47) CometProject Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -305,158 +296,156 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter +(49) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(52) CometProject +(50) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(53) CometBroadcastExchange +(51) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(56) CometProject +(54) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(57) CometBroadcastHashJoin +(55) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(58) CometProject +(56) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(59) ReusedExchange [Reuses operator id: 13] +(57) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(60) CometBroadcastHashJoin +(58) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(61) CometProject +(59) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(62) CometHashAggregate +(60) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(61) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(64) CometHashAggregate +(62) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(65) CometExchange +(63) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) CometSort +(64) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) CometColumnarToRow [codegen id : 7] +(65) CometWindowExec Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] +(66) CometSort Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] -(70) Window +(67) CometWindowExec Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(71) Filter [codegen id : 9] +(68) CometFilter Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(69) CometProject Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(73) Union +(70) CometUnion +Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(74) HashAggregate [codegen id : 10] +(71) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarExchange +(72) CometExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(76) CometHashAggregate +(73) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(77) CometTakeOrderedAndProject +(74) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(78) CometColumnarToRow [codegen id : 11] +(75) CometColumnarToRow [codegen id : 1] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter +(77) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(81) CometProject +(78) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(79) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(83) BroadcastExchange +(80) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt index 4988a26564..6a8de7fe28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt @@ -1,94 +1,91 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt index ca80833ee5..4f128e83de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt @@ -1,110 +1,87 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometUnion [channel,item,return_ratio,return_rank,currency_rank] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index 74702d596d..2c993b48e0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -1,82 +1,79 @@ == Physical Plan == -* CometColumnarToRow (78) -+- CometTakeOrderedAndProject (77) - +- CometHashAggregate (76) - +- CometColumnarExchange (75) - +- * HashAggregate (74) - +- Union (73) - :- * Project (26) - : +- * Filter (25) - : +- Window (24) - : +- * Sort (23) - : +- Window (22) - : +- * CometColumnarToRow (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- * Project (49) - : +- * Filter (48) - : +- Window (47) - : +- * Sort (46) - : +- Window (45) - : +- * CometColumnarToRow (44) - : +- CometSort (43) - : +- CometExchange (42) - : +- CometHashAggregate (41) - : +- CometExchange (40) - : +- CometHashAggregate (39) - : +- CometProject (38) - : +- CometBroadcastHashJoin (37) - : :- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometBroadcastExchange (30) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) - : : +- CometProject (33) - : : +- CometFilter (32) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) - : +- ReusedExchange (36) - +- * Project (72) - +- * Filter (71) - +- Window (70) - +- * Sort (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometProject (61) - +- CometBroadcastHashJoin (60) - :- CometProject (58) - : +- CometBroadcastHashJoin (57) - : :- CometBroadcastExchange (53) - : : +- CometProject (52) - : : +- CometFilter (51) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) - +- ReusedExchange (59) +* CometColumnarToRow (75) ++- CometTakeOrderedAndProject (74) + +- CometHashAggregate (73) + +- CometExchange (72) + +- CometHashAggregate (71) + +- CometUnion (70) + :- CometProject (25) + : +- CometFilter (24) + : +- CometWindowExec (23) + : +- CometSort (22) + : +- CometWindowExec (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- CometProject (47) + : +- CometFilter (46) + : +- CometWindowExec (45) + : +- CometSort (44) + : +- CometWindowExec (43) + : +- CometSort (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometExchange (39) + : +- CometHashAggregate (38) + : +- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (34) + : : +- CometBroadcastHashJoin (33) + : : :- CometBroadcastExchange (29) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) + : +- ReusedExchange (35) + +- CometProject (69) + +- CometFilter (68) + +- CometWindowExec (67) + +- CometSort (66) + +- CometWindowExec (65) + +- CometSort (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometExchange (61) + +- CometHashAggregate (60) + +- CometProject (59) + +- CometBroadcastHashJoin (58) + :- CometProject (56) + : +- CometBroadcastHashJoin (55) + : :- CometBroadcastExchange (51) + : : +- CometProject (50) + : : +- CometFilter (49) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) + +- ReusedExchange (57) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -173,30 +170,27 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometColumnarToRow [codegen id : 1] +(21) CometWindowExec Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) Window -Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] - -(23) Sort [codegen id : 2] +(22) CometSort Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] -(24) Window +(23) CometWindowExec Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(25) Filter [codegen id : 3] +(24) CometFilter Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(26) Project [codegen id : 3] -Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +(25) CometProject Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] +Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -204,100 +198,97 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(28) CometFilter +(27) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(29) CometProject +(28) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometBroadcastExchange +(29) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(32) CometFilter +(31) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(33) CometProject +(32) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(34) CometBroadcastHashJoin +(33) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(35) CometProject +(34) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(36) ReusedExchange [Reuses operator id: 13] +(35) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(37) CometBroadcastHashJoin +(36) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(38) CometProject +(37) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(39) CometHashAggregate +(38) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(40) CometExchange +(39) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(41) CometHashAggregate +(40) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(42) CometExchange +(41) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort +(42) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(44) CometColumnarToRow [codegen id : 4] -Input [3]: [item#47, return_ratio#48, currency_ratio#49] - -(45) Window +(43) CometWindowExec Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(46) Sort [codegen id : 5] +(44) CometSort Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] -(47) Window +(45) CometWindowExec Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(48) Filter [codegen id : 6] +(46) CometFilter Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(49) Project [codegen id : 6] -Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +(47) CometProject Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] +Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -305,158 +296,156 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(51) CometFilter +(49) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(52) CometProject +(50) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(53) CometBroadcastExchange +(51) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(55) CometFilter +(53) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(56) CometProject +(54) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(57) CometBroadcastHashJoin +(55) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(58) CometProject +(56) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(59) ReusedExchange [Reuses operator id: 13] +(57) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(60) CometBroadcastHashJoin +(58) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(61) CometProject +(59) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(62) CometHashAggregate +(60) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(61) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(64) CometHashAggregate +(62) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(65) CometExchange +(63) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(66) CometSort +(64) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(67) CometColumnarToRow [codegen id : 7] +(65) CometWindowExec Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(68) Window -Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] - -(69) Sort [codegen id : 8] +(66) CometSort Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] -(70) Window +(67) CometWindowExec Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(71) Filter [codegen id : 9] +(68) CometFilter Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(72) Project [codegen id : 9] -Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(69) CometProject Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] +Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(73) Union +(70) CometUnion +Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] +Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] +Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(74) HashAggregate [codegen id : 10] +(71) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarExchange +(72) CometExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(76) CometHashAggregate +(73) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(77) CometTakeOrderedAndProject +(74) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(78) CometColumnarToRow [codegen id : 11] +(75) CometColumnarToRow [codegen id : 1] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (83) -+- * CometColumnarToRow (82) - +- CometProject (81) - +- CometFilter (80) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) +BroadcastExchange (80) ++- * CometColumnarToRow (79) + +- CometProject (78) + +- CometFilter (77) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) -(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(80) CometFilter +(77) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(81) CometProject +(78) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(82) CometColumnarToRow [codegen id : 1] +(79) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(83) BroadcastExchange +(80) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt index 4988a26564..6a8de7fe28 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt @@ -1,94 +1,91 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- Project - : +- Filter - : +- Window - : +- Sort - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- Project - +- Filter - +- Window - +- Sort - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index ca80833ee5..4f128e83de 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -1,110 +1,87 @@ -WholeStageCodegen (11) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - WholeStageCodegen (10) - HashAggregate [channel,item,return_ratio,return_rank,currency_rank] - InputAdapter - Union - WholeStageCodegen (3) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (2) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - WholeStageCodegen (6) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (5) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - WholeStageCodegen (9) - Project [item,return_ratio,return_rank,currency_rank] - Filter [return_rank,currency_rank] - InputAdapter - Window [currency_ratio] - WholeStageCodegen (8) - Sort [currency_ratio] - InputAdapter - Window [return_ratio] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] + CometUnion [channel,item,return_ratio,return_rank,currency_rank] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometProject [channel,item,return_ratio,return_rank,currency_rank] + CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] + CometSort [item,return_ratio,currency_ratio,return_rank] + CometWindowExec [item,return_ratio,currency_ratio,return_rank] + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt index f8a95a6db8..ce43cefb57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt @@ -1,80 +1,73 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * Filter (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- Window (64) - : +- * CometColumnarToRow (63) - : +- CometSort (62) - : +- CometExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometSortMergeJoin (58) - : :- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * CometColumnarToRow (27) - : : +- CometColumnarExchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * CometColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- Window (20) - : : +- * CometColumnarToRow (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (57) - : +- CometColumnarExchange (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (43) - : : +- Window (42) - : : +- * CometColumnarToRow (41) - : : +- CometSort (40) - : : +- CometExchange (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometFilter (32) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : +- ReusedExchange (33) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- Window (47) - : +- * CometColumnarToRow (46) - : +- CometSort (45) - : +- ReusedExchange (44) - +- BroadcastExchange (70) - +- * Project (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- ReusedExchange (65) +* CometColumnarToRow (69) ++- CometTakeOrderedAndProject (68) + +- CometFilter (67) + +- CometHashAggregate (66) + +- CometHashAggregate (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometWindowExec (57) + : +- CometSort (56) + : +- CometExchange (55) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometSortMergeJoin (52) + : :- CometSort (27) + : : +- CometExchange (26) + : : +- CometHashAggregate (25) + : : +- CometExchange (24) + : : +- CometHashAggregate (23) + : : +- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometProject (15) + : : : +- CometWindowExec (14) + : : : +- CometSort (13) + : : : +- CometExchange (12) + : : : +- CometHashAggregate (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (20) + : : +- CometProject (19) + : : +- CometWindowExec (18) + : : +- CometSort (17) + : : +- ReusedExchange (16) + : +- CometSort (51) + : +- CometExchange (50) + : +- CometHashAggregate (49) + : +- CometExchange (48) + : +- CometHashAggregate (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (39) + : : +- CometWindowExec (38) + : : +- CometSort (37) + : : +- CometExchange (36) + : : +- CometHashAggregate (35) + : : +- CometExchange (34) + : : +- CometHashAggregate (33) + : : +- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometFilter (29) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) + : : +- ReusedExchange (30) + : +- CometBroadcastExchange (44) + : +- CometProject (43) + : +- CometWindowExec (42) + : +- CometSort (41) + : +- ReusedExchange (40) + +- CometBroadcastExchange (62) + +- CometProject (61) + +- CometWindowExec (60) + +- CometSort (59) + +- ReusedExchange (58) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -139,310 +132,274 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) CometColumnarToRow [codegen id : 1] +(14) CometWindowExec Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11], [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Window -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] +(15) CometProject Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] +Arguments: [item_sk#9, d_date#6, sumws#10, rk#11], [item_sk#9, d_date#6, sumws#10, rk#11] -(17) ReusedExchange [Reuses operator id: 12] +(16) ReusedExchange [Reuses operator id: 12] Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(18) CometSort +(17) CometSort Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] -(19) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] - -(20) Window +(18) CometWindowExec Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] +Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14], [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] -(21) Project [codegen id : 3] -Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] +(19) CometProject Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] +Arguments: [item_sk#15, sumws#16, rk#14], [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] -(22) BroadcastExchange +(20) CometBroadcastExchange Input [3]: [item_sk#15, sumws#16, rk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [item_sk#15, sumws#16, rk#14] -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#9] -Right keys [1]: [item_sk#15] -Join type: Inner -Join condition: (rk#11 >= rk#14) +(21) CometBroadcastHashJoin +Left output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] +Right output [3]: [item_sk#15, sumws#16, rk#14] +Arguments: [item_sk#9], [item_sk#15], Inner, (rk#11 >= rk#14), BuildRight -(24) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +(22) CometProject Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] +Arguments: [item_sk#9, d_date#6, sumws#10, sumws#16], [item_sk#9, d_date#6, sumws#10, sumws#16] -(25) HashAggregate [codegen id : 4] +(23) CometHashAggregate Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] Keys [3]: [item_sk#9, d_date#6, sumws#10] Functions [1]: [partial_sum(sumws#16)] -Aggregate Attributes [2]: [sum#17, isEmpty#18] -Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(26) CometColumnarExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(27) CometColumnarToRow [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +(24) CometExchange +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#17, isEmpty#18] +Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(28) HashAggregate [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +(25) CometHashAggregate +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#17, isEmpty#18] Keys [3]: [item_sk#9, d_date#6, sumws#10] Functions [1]: [sum(sumws#16)] -Aggregate Attributes [1]: [sum(sumws#16)#21] -Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] -(29) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(26) CometExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#19] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(30) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(27) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#19] +Arguments: [item_sk#9, d_date#6, cume_sales#19], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_item_sk#23) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#27, d_date#28] - -(34) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Right output [2]: [d_date_sk#27, d_date#28] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(35) CometProject -Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] -Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] - -(36) CometHashAggregate -Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] - -(37) CometExchange -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(38) CometHashAggregate -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] - -(39) CometExchange -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(29) CometFilter +Input [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_item_sk#20) + +(30) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#24, d_date#25] + +(31) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] +Right output [2]: [d_date_sk#24, d_date#25] +Arguments: [ss_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight + +(32) CometProject +Input [5]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#24, d_date#25] +Arguments: [ss_item_sk#20, ss_sales_price#21, d_date#25], [ss_item_sk#20, ss_sales_price#21, d_date#25] + +(33) CometHashAggregate +Input [3]: [ss_item_sk#20, ss_sales_price#21, d_date#25] +Keys [2]: [ss_item_sk#20, d_date#25] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] + +(34) CometExchange +Input [3]: [ss_item_sk#20, d_date#25, sum#26] +Arguments: hashpartitioning(ss_item_sk#20, d_date#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(35) CometHashAggregate +Input [3]: [ss_item_sk#20, d_date#25, sum#26] +Keys [2]: [ss_item_sk#20, d_date#25] +Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] + +(36) CometExchange +Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] +Arguments: hashpartitioning(ss_item_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(37) CometSort +Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] +Arguments: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20], [ss_item_sk#20 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] + +(38) CometWindowExec +Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] +Arguments: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20, rk#29], [row_number() windowspecdefinition(ss_item_sk#20, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#29], [ss_item_sk#20], [d_date#25 ASC NULLS FIRST] + +(39) CometProject +Input [5]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20, rk#29] +Arguments: [item_sk#27, d_date#25, sumss#28, rk#29], [item_sk#27, d_date#25, sumss#28, rk#29] + +(40) ReusedExchange [Reuses operator id: 36] +Output [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] + +(41) CometSort +Input [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] +Arguments: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31], [ss_item_sk#31 ASC NULLS FIRST, d_date#30 ASC NULLS FIRST] + +(42) CometWindowExec +Input [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] +Arguments: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31, rk#32], [row_number() windowspecdefinition(ss_item_sk#31, d_date#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#31], [d_date#30 ASC NULLS FIRST] + +(43) CometProject +Input [5]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31, rk#32] +Arguments: [item_sk#33, sumss#34, rk#32], [item_sk#27 AS item_sk#33, sumss#28 AS sumss#34, rk#32] + +(44) CometBroadcastExchange +Input [3]: [item_sk#33, sumss#34, rk#32] +Arguments: [item_sk#33, sumss#34, rk#32] + +(45) CometBroadcastHashJoin +Left output [4]: [item_sk#27, d_date#25, sumss#28, rk#29] +Right output [3]: [item_sk#33, sumss#34, rk#32] +Arguments: [item_sk#27], [item_sk#33], Inner, (rk#29 >= rk#32), BuildRight + +(46) CometProject +Input [7]: [item_sk#27, d_date#25, sumss#28, rk#29, item_sk#33, sumss#34, rk#32] +Arguments: [item_sk#27, d_date#25, sumss#28, sumss#34], [item_sk#27, d_date#25, sumss#28, sumss#34] + +(47) CometHashAggregate +Input [4]: [item_sk#27, d_date#25, sumss#28, sumss#34] +Keys [3]: [item_sk#27, d_date#25, sumss#28] +Functions [1]: [partial_sum(sumss#34)] + +(48) CometExchange +Input [5]: [item_sk#27, d_date#25, sumss#28, sum#35, isEmpty#36] +Arguments: hashpartitioning(item_sk#27, d_date#25, sumss#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(49) CometHashAggregate +Input [5]: [item_sk#27, d_date#25, sumss#28, sum#35, isEmpty#36] +Keys [3]: [item_sk#27, d_date#25, sumss#28] +Functions [1]: [sum(sumss#34)] + +(50) CometExchange +Input [3]: [item_sk#27, d_date#25, cume_sales#37] +Arguments: hashpartitioning(item_sk#27, d_date#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(51) CometSort +Input [3]: [item_sk#27, d_date#25, cume_sales#37] +Arguments: [item_sk#27, d_date#25, cume_sales#37], [item_sk#27 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] + +(52) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#19] +Right output [3]: [item_sk#27, d_date#25, cume_sales#37] +Arguments: [item_sk#9, d_date#6], [item_sk#27, d_date#25], FullOuter + +(53) CometFilter +Input [6]: [item_sk#9, d_date#6, cume_sales#19, item_sk#27, d_date#25, cume_sales#37] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#27 END) + +(54) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#19, item_sk#27, d_date#25, cume_sales#37] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#27 END AS item_sk#38, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#25 END AS d_date#39, cume_sales#19 AS web_sales#40, cume_sales#37 AS store_sales#41] + +(55) CometExchange +Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Arguments: hashpartitioning(item_sk#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] + +(56) CometSort +Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST] + +(57) CometWindowExec +Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42], [row_number() windowspecdefinition(item_sk#38, d_date#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [item_sk#38], [d_date#39 ASC NULLS FIRST] + +(58) ReusedExchange [Reuses operator id: 55] +Output [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -(40) CometSort -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] +(59) CometSort +Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST] -(41) CometColumnarToRow [codegen id : 6] -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +(60) CometWindowExec +Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#43], [row_number() windowspecdefinition(item_sk#38, d_date#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [item_sk#38], [d_date#39 ASC NULLS FIRST] -(42) Window -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] +(61) CometProject +Input [5]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#43] +Arguments: [item_sk#44, web_sales#45, store_sales#46, rk#43], [item_sk#38 AS item_sk#44, web_sales#40 AS web_sales#45, store_sales#41 AS store_sales#46, rk#43] -(43) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] -Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] +(62) CometBroadcastExchange +Input [4]: [item_sk#44, web_sales#45, store_sales#46, rk#43] +Arguments: [item_sk#44, web_sales#45, store_sales#46, rk#43] -(44) ReusedExchange [Reuses operator id: 39] -Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +(63) CometBroadcastHashJoin +Left output [5]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42] +Right output [4]: [item_sk#44, web_sales#45, store_sales#46, rk#43] +Arguments: [item_sk#38], [item_sk#44], Inner, (rk#42 >= rk#43), BuildRight -(45) CometSort -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] +(64) CometProject +Input [9]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42, item_sk#44, web_sales#45, store_sales#46, rk#43] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46], [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46] -(46) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] - -(47) Window -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] - -(48) Project [codegen id : 8] -Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] -Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] - -(49) BroadcastExchange -Input [3]: [item_sk#36, sumss#37, rk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(50) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#30] -Right keys [1]: [item_sk#36] -Join type: Inner -Join condition: (rk#32 >= rk#35) - -(51) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] - -(52) HashAggregate [codegen id : 9] -Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(53) CometColumnarExchange -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(54) CometColumnarToRow [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(55) HashAggregate [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#42] -Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] - -(56) CometColumnarExchange -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(57) CometSort -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#22] -Right output [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter - -(59) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) - -(60) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] - -(61) CometExchange -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(62) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(63) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(64) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(65) ReusedExchange [Reuses operator id: 61] -Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(66) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 22] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(68) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(69) Project [codegen id : 23] -Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] -Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] - -(70) BroadcastExchange -Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [item_sk#44] -Right keys [1]: [item_sk#50] -Join type: Inner -Join condition: (rk#48 >= rk#49) - -(72) Project [codegen id : 24] -Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] - -(73) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] -Aggregate Attributes [2]: [max#53, max#54] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] - -(74) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [max(web_sales#51), max(store_sales#52)] -Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] - -(75) Filter [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) - -(76) TakeOrderedAndProject -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +(65) CometHashAggregate +Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46] +Keys [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Functions [2]: [partial_max(web_sales#45), partial_max(store_sales#46)] + +(66) CometHashAggregate +Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, max#47, max#48] +Keys [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Functions [2]: [max(web_sales#45), max(store_sales#46)] + +(67) CometFilter +Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] +Condition : ((isnotnull(web_cumulative#49) AND isnotnull(store_cumulative#50)) AND (web_cumulative#49 > store_cumulative#50)) + +(68) CometTakeOrderedAndProject +Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#38 ASC NULLS FIRST,d_date#39 ASC NULLS FIRST], output=[item_sk#38,d_date#39,web_sales#40,store_sales#41,web_cumulative#49,store_cumulative#50]), [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50], 100, 0, [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST], [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] + +(69) CometColumnarToRow [codegen id : 1] +Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +BroadcastExchange (74) ++- * CometColumnarToRow (73) + +- CometProject (72) + +- CometFilter (71) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (70) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(78) CometFilter +(71) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(79) CometProject +(72) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(80) CometColumnarToRow [codegen id : 1] +(73) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(81) BroadcastExchange +(74) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt index 86caa88151..8aed2793f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt @@ -1,216 +1,203 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometWindowExec + : : : +- CometSort + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 184 out of 196 eligible operators (93%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt index b3013059b0..09f0ec24d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt @@ -1,116 +1,80 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (24) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometHashAggregate [max,max] [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative,max(web_sales),max(store_sales)] + CometHashAggregate [web_sales,store_sales] [item_sk,d_date,web_sales,store_sales,max,max] + CometProject [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + CometBroadcastHashJoin [item_sk,d_date,web_sales,store_sales,rk,item_sk,web_sales,store_sales,rk] + CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #2 + CometHashAggregate [sumws,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumws)] + CometExchange [item_sk,d_date,sumws] #3 + CometHashAggregate [sumws] [item_sk,d_date,sumws,sum,isEmpty] + CometProject [item_sk,d_date,sumws,sumws] + CometBroadcastHashJoin [item_sk,d_date,sumws,rk,item_sk,sumws,rk] + CometProject [item_sk,d_date,sumws,rk] + CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometExchange [ws_item_sk] #4 + CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [item_sk,sumws,rk] #8 + CometProject [item_sk,sumws] [item_sk,sumws,rk] + CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #9 + CometHashAggregate [sumss,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumss)] + CometExchange [item_sk,d_date,sumss] #10 + CometHashAggregate [sumss] [item_sk,d_date,sumss,sum,isEmpty] + CometProject [item_sk,d_date,sumss,sumss] + CometBroadcastHashJoin [item_sk,d_date,sumss,rk,item_sk,sumss,rk] + CometProject [item_sk,d_date,sumss,rk] + CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 + CometBroadcastExchange [item_sk,sumss,rk] #13 + CometProject [item_sk,sumss] [item_sk,sumss,rk] + CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + CometBroadcastExchange [item_sk,web_sales,store_sales,rk] #14 + CometProject [item_sk,web_sales,store_sales] [item_sk,web_sales,store_sales,rk] + CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (4) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (9) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index f8a95a6db8..ce43cefb57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -1,80 +1,73 @@ == Physical Plan == -TakeOrderedAndProject (76) -+- * Filter (75) - +- * HashAggregate (74) - +- * HashAggregate (73) - +- * Project (72) - +- * BroadcastHashJoin Inner BuildRight (71) - :- Window (64) - : +- * CometColumnarToRow (63) - : +- CometSort (62) - : +- CometExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometSortMergeJoin (58) - : :- CometSort (30) - : : +- CometColumnarExchange (29) - : : +- * HashAggregate (28) - : : +- * CometColumnarToRow (27) - : : +- CometColumnarExchange (26) - : : +- * HashAggregate (25) - : : +- * Project (24) - : : +- * BroadcastHashJoin Inner BuildRight (23) - : : :- * Project (16) - : : : +- Window (15) - : : : +- * CometColumnarToRow (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- BroadcastExchange (22) - : : +- * Project (21) - : : +- Window (20) - : : +- * CometColumnarToRow (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (57) - : +- CometColumnarExchange (56) - : +- * HashAggregate (55) - : +- * CometColumnarToRow (54) - : +- CometColumnarExchange (53) - : +- * HashAggregate (52) - : +- * Project (51) - : +- * BroadcastHashJoin Inner BuildRight (50) - : :- * Project (43) - : : +- Window (42) - : : +- * CometColumnarToRow (41) - : : +- CometSort (40) - : : +- CometExchange (39) - : : +- CometHashAggregate (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometFilter (32) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) - : : +- ReusedExchange (33) - : +- BroadcastExchange (49) - : +- * Project (48) - : +- Window (47) - : +- * CometColumnarToRow (46) - : +- CometSort (45) - : +- ReusedExchange (44) - +- BroadcastExchange (70) - +- * Project (69) - +- Window (68) - +- * CometColumnarToRow (67) - +- CometSort (66) - +- ReusedExchange (65) +* CometColumnarToRow (69) ++- CometTakeOrderedAndProject (68) + +- CometFilter (67) + +- CometHashAggregate (66) + +- CometHashAggregate (65) + +- CometProject (64) + +- CometBroadcastHashJoin (63) + :- CometWindowExec (57) + : +- CometSort (56) + : +- CometExchange (55) + : +- CometProject (54) + : +- CometFilter (53) + : +- CometSortMergeJoin (52) + : :- CometSort (27) + : : +- CometExchange (26) + : : +- CometHashAggregate (25) + : : +- CometExchange (24) + : : +- CometHashAggregate (23) + : : +- CometProject (22) + : : +- CometBroadcastHashJoin (21) + : : :- CometProject (15) + : : : +- CometWindowExec (14) + : : : +- CometSort (13) + : : : +- CometExchange (12) + : : : +- CometHashAggregate (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- CometBroadcastExchange (20) + : : +- CometProject (19) + : : +- CometWindowExec (18) + : : +- CometSort (17) + : : +- ReusedExchange (16) + : +- CometSort (51) + : +- CometExchange (50) + : +- CometHashAggregate (49) + : +- CometExchange (48) + : +- CometHashAggregate (47) + : +- CometProject (46) + : +- CometBroadcastHashJoin (45) + : :- CometProject (39) + : : +- CometWindowExec (38) + : : +- CometSort (37) + : : +- CometExchange (36) + : : +- CometHashAggregate (35) + : : +- CometExchange (34) + : : +- CometHashAggregate (33) + : : +- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometFilter (29) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) + : : +- ReusedExchange (30) + : +- CometBroadcastExchange (44) + : +- CometProject (43) + : +- CometWindowExec (42) + : +- CometSort (41) + : +- ReusedExchange (40) + +- CometBroadcastExchange (62) + +- CometProject (61) + +- CometWindowExec (60) + +- CometSort (59) + +- ReusedExchange (58) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -139,310 +132,274 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) CometColumnarToRow [codegen id : 1] +(14) CometWindowExec Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11], [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) Window -Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] - -(16) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] +(15) CometProject Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] +Arguments: [item_sk#9, d_date#6, sumws#10, rk#11], [item_sk#9, d_date#6, sumws#10, rk#11] -(17) ReusedExchange [Reuses operator id: 12] +(16) ReusedExchange [Reuses operator id: 12] Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(18) CometSort +(17) CometSort Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] -(19) CometColumnarToRow [codegen id : 2] -Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] - -(20) Window +(18) CometWindowExec Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] +Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14], [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] -(21) Project [codegen id : 3] -Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] +(19) CometProject Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] +Arguments: [item_sk#15, sumws#16, rk#14], [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] -(22) BroadcastExchange +(20) CometBroadcastExchange Input [3]: [item_sk#15, sumws#16, rk#14] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [item_sk#15, sumws#16, rk#14] -(23) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [item_sk#9] -Right keys [1]: [item_sk#15] -Join type: Inner -Join condition: (rk#11 >= rk#14) +(21) CometBroadcastHashJoin +Left output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] +Right output [3]: [item_sk#15, sumws#16, rk#14] +Arguments: [item_sk#9], [item_sk#15], Inner, (rk#11 >= rk#14), BuildRight -(24) Project [codegen id : 4] -Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] +(22) CometProject Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] +Arguments: [item_sk#9, d_date#6, sumws#10, sumws#16], [item_sk#9, d_date#6, sumws#10, sumws#16] -(25) HashAggregate [codegen id : 4] +(23) CometHashAggregate Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] Keys [3]: [item_sk#9, d_date#6, sumws#10] Functions [1]: [partial_sum(sumws#16)] -Aggregate Attributes [2]: [sum#17, isEmpty#18] -Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] - -(26) CometColumnarExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(27) CometColumnarToRow [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +(24) CometExchange +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#17, isEmpty#18] +Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(28) HashAggregate [codegen id : 5] -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +(25) CometHashAggregate +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#17, isEmpty#18] Keys [3]: [item_sk#9, d_date#6, sumws#10] Functions [1]: [sum(sumws#16)] -Aggregate Attributes [1]: [sum(sumws#16)#21] -Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] -(29) CometColumnarExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(26) CometExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#19] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(30) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#22] -Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(27) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#19] +Arguments: [item_sk#9, d_date#6, cume_sales#19], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] +PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(32) CometFilter -Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Condition : isnotnull(ss_item_sk#23) - -(33) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#27, d_date#28] - -(34) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] -Right output [2]: [d_date_sk#27, d_date#28] -Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight - -(35) CometProject -Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] -Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] - -(36) CometHashAggregate -Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] - -(37) CometExchange -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(38) CometHashAggregate -Input [3]: [ss_item_sk#23, d_date#28, sum#29] -Keys [2]: [ss_item_sk#23, d_date#28] -Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] - -(39) CometExchange -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(29) CometFilter +Input [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] +Condition : isnotnull(ss_item_sk#20) + +(30) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#24, d_date#25] + +(31) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] +Right output [2]: [d_date_sk#24, d_date#25] +Arguments: [ss_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight + +(32) CometProject +Input [5]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#24, d_date#25] +Arguments: [ss_item_sk#20, ss_sales_price#21, d_date#25], [ss_item_sk#20, ss_sales_price#21, d_date#25] + +(33) CometHashAggregate +Input [3]: [ss_item_sk#20, ss_sales_price#21, d_date#25] +Keys [2]: [ss_item_sk#20, d_date#25] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] + +(34) CometExchange +Input [3]: [ss_item_sk#20, d_date#25, sum#26] +Arguments: hashpartitioning(ss_item_sk#20, d_date#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(35) CometHashAggregate +Input [3]: [ss_item_sk#20, d_date#25, sum#26] +Keys [2]: [ss_item_sk#20, d_date#25] +Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] + +(36) CometExchange +Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] +Arguments: hashpartitioning(ss_item_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(37) CometSort +Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] +Arguments: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20], [ss_item_sk#20 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] + +(38) CometWindowExec +Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] +Arguments: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20, rk#29], [row_number() windowspecdefinition(ss_item_sk#20, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#29], [ss_item_sk#20], [d_date#25 ASC NULLS FIRST] + +(39) CometProject +Input [5]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20, rk#29] +Arguments: [item_sk#27, d_date#25, sumss#28, rk#29], [item_sk#27, d_date#25, sumss#28, rk#29] + +(40) ReusedExchange [Reuses operator id: 36] +Output [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] + +(41) CometSort +Input [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] +Arguments: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31], [ss_item_sk#31 ASC NULLS FIRST, d_date#30 ASC NULLS FIRST] + +(42) CometWindowExec +Input [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] +Arguments: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31, rk#32], [row_number() windowspecdefinition(ss_item_sk#31, d_date#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#31], [d_date#30 ASC NULLS FIRST] + +(43) CometProject +Input [5]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31, rk#32] +Arguments: [item_sk#33, sumss#34, rk#32], [item_sk#27 AS item_sk#33, sumss#28 AS sumss#34, rk#32] + +(44) CometBroadcastExchange +Input [3]: [item_sk#33, sumss#34, rk#32] +Arguments: [item_sk#33, sumss#34, rk#32] + +(45) CometBroadcastHashJoin +Left output [4]: [item_sk#27, d_date#25, sumss#28, rk#29] +Right output [3]: [item_sk#33, sumss#34, rk#32] +Arguments: [item_sk#27], [item_sk#33], Inner, (rk#29 >= rk#32), BuildRight + +(46) CometProject +Input [7]: [item_sk#27, d_date#25, sumss#28, rk#29, item_sk#33, sumss#34, rk#32] +Arguments: [item_sk#27, d_date#25, sumss#28, sumss#34], [item_sk#27, d_date#25, sumss#28, sumss#34] + +(47) CometHashAggregate +Input [4]: [item_sk#27, d_date#25, sumss#28, sumss#34] +Keys [3]: [item_sk#27, d_date#25, sumss#28] +Functions [1]: [partial_sum(sumss#34)] + +(48) CometExchange +Input [5]: [item_sk#27, d_date#25, sumss#28, sum#35, isEmpty#36] +Arguments: hashpartitioning(item_sk#27, d_date#25, sumss#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] + +(49) CometHashAggregate +Input [5]: [item_sk#27, d_date#25, sumss#28, sum#35, isEmpty#36] +Keys [3]: [item_sk#27, d_date#25, sumss#28] +Functions [1]: [sum(sumss#34)] + +(50) CometExchange +Input [3]: [item_sk#27, d_date#25, cume_sales#37] +Arguments: hashpartitioning(item_sk#27, d_date#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] + +(51) CometSort +Input [3]: [item_sk#27, d_date#25, cume_sales#37] +Arguments: [item_sk#27, d_date#25, cume_sales#37], [item_sk#27 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] + +(52) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#19] +Right output [3]: [item_sk#27, d_date#25, cume_sales#37] +Arguments: [item_sk#9, d_date#6], [item_sk#27, d_date#25], FullOuter + +(53) CometFilter +Input [6]: [item_sk#9, d_date#6, cume_sales#19, item_sk#27, d_date#25, cume_sales#37] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#27 END) + +(54) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#19, item_sk#27, d_date#25, cume_sales#37] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#27 END AS item_sk#38, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#25 END AS d_date#39, cume_sales#19 AS web_sales#40, cume_sales#37 AS store_sales#41] + +(55) CometExchange +Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Arguments: hashpartitioning(item_sk#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] + +(56) CometSort +Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST] + +(57) CometWindowExec +Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42], [row_number() windowspecdefinition(item_sk#38, d_date#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [item_sk#38], [d_date#39 ASC NULLS FIRST] + +(58) ReusedExchange [Reuses operator id: 55] +Output [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -(40) CometSort -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] +(59) CometSort +Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST] -(41) CometColumnarToRow [codegen id : 6] -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +(60) CometWindowExec +Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#43], [row_number() windowspecdefinition(item_sk#38, d_date#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [item_sk#38], [d_date#39 ASC NULLS FIRST] -(42) Window -Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] +(61) CometProject +Input [5]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#43] +Arguments: [item_sk#44, web_sales#45, store_sales#46, rk#43], [item_sk#38 AS item_sk#44, web_sales#40 AS web_sales#45, store_sales#41 AS store_sales#46, rk#43] -(43) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] -Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] +(62) CometBroadcastExchange +Input [4]: [item_sk#44, web_sales#45, store_sales#46, rk#43] +Arguments: [item_sk#44, web_sales#45, store_sales#46, rk#43] -(44) ReusedExchange [Reuses operator id: 39] -Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +(63) CometBroadcastHashJoin +Left output [5]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42] +Right output [4]: [item_sk#44, web_sales#45, store_sales#46, rk#43] +Arguments: [item_sk#38], [item_sk#44], Inner, (rk#42 >= rk#43), BuildRight -(45) CometSort -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] +(64) CometProject +Input [9]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42, item_sk#44, web_sales#45, store_sales#46, rk#43] +Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46], [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46] -(46) CometColumnarToRow [codegen id : 7] -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] - -(47) Window -Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] - -(48) Project [codegen id : 8] -Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] -Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] - -(49) BroadcastExchange -Input [3]: [item_sk#36, sumss#37, rk#35] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] - -(50) BroadcastHashJoin [codegen id : 9] -Left keys [1]: [item_sk#30] -Right keys [1]: [item_sk#36] -Join type: Inner -Join condition: (rk#32 >= rk#35) - -(51) Project [codegen id : 9] -Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] - -(52) HashAggregate [codegen id : 9] -Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [partial_sum(sumss#37)] -Aggregate Attributes [2]: [sum#38, isEmpty#39] -Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(53) CometColumnarExchange -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] - -(54) CometColumnarToRow [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] - -(55) HashAggregate [codegen id : 10] -Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] -Keys [3]: [item_sk#30, d_date#28, sumss#31] -Functions [1]: [sum(sumss#37)] -Aggregate Attributes [1]: [sum(sumss#37)#42] -Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] - -(56) CometColumnarExchange -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(57) CometSort -Input [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] - -(58) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#22] -Right output [3]: [item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter - -(59) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) - -(60) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] - -(61) CometExchange -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] - -(62) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(63) CometColumnarToRow [codegen id : 11] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(64) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(65) ReusedExchange [Reuses operator id: 61] -Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(66) CometSort -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] - -(67) CometColumnarToRow [codegen id : 22] -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] - -(68) Window -Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] - -(69) Project [codegen id : 23] -Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] -Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] - -(70) BroadcastExchange -Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] - -(71) BroadcastHashJoin [codegen id : 24] -Left keys [1]: [item_sk#44] -Right keys [1]: [item_sk#50] -Join type: Inner -Join condition: (rk#48 >= rk#49) - -(72) Project [codegen id : 24] -Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] - -(73) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] -Aggregate Attributes [2]: [max#53, max#54] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] - -(74) HashAggregate [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] -Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] -Functions [2]: [max(web_sales#51), max(store_sales#52)] -Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] -Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] - -(75) Filter [codegen id : 24] -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) - -(76) TakeOrderedAndProject -Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] -Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +(65) CometHashAggregate +Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46] +Keys [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Functions [2]: [partial_max(web_sales#45), partial_max(store_sales#46)] + +(66) CometHashAggregate +Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, max#47, max#48] +Keys [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +Functions [2]: [max(web_sales#45), max(store_sales#46)] + +(67) CometFilter +Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] +Condition : ((isnotnull(web_cumulative#49) AND isnotnull(store_cumulative#50)) AND (web_cumulative#49 > store_cumulative#50)) + +(68) CometTakeOrderedAndProject +Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#38 ASC NULLS FIRST,d_date#39 ASC NULLS FIRST], output=[item_sk#38,d_date#39,web_sales#40,store_sales#41,web_cumulative#49,store_cumulative#50]), [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50], 100, 0, [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST], [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] + +(69) CometColumnarToRow [codegen id : 1] +Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (81) -+- * CometColumnarToRow (80) - +- CometProject (79) - +- CometFilter (78) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) +BroadcastExchange (74) ++- * CometColumnarToRow (73) + +- CometProject (72) + +- CometFilter (71) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (70) -(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(78) CometFilter +(71) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(79) CometProject +(72) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(80) CometColumnarToRow [codegen id : 1] +(73) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(81) BroadcastExchange +(74) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] -Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt index 86caa88151..8aed2793f6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt @@ -1,216 +1,203 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- HashAggregate - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- CometColumnarToRow - : : +- CometColumnarExchange - : : +- HashAggregate - : : +- Project - : : +- BroadcastHashJoin - : : :- Project - : : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : : +- CometColumnarToRow - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- BroadcastExchange - : : +- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometHashAggregate + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometWindowExec + : : : +- CometSort + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometSortMergeJoin :- CometSort - : +- CometColumnarExchange - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate : +- CometExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometColumnarExchange - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 184 out of 196 eligible operators (93%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index b3013059b0..09f0ec24d0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -1,116 +1,80 @@ -TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - WholeStageCodegen (24) - Filter [web_cumulative,store_cumulative] - HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] - HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] - Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (11) - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + CometHashAggregate [max,max] [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative,max(web_sales),max(store_sales)] + CometHashAggregate [web_sales,store_sales] [item_sk,d_date,web_sales,store_sales,max,max] + CometProject [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + CometBroadcastHashJoin [item_sk,d_date,web_sales,store_sales,rk,item_sk,web_sales,store_sales,rk] + CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #2 + CometHashAggregate [sumws,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumws)] + CometExchange [item_sk,d_date,sumws] #3 + CometHashAggregate [sumws] [item_sk,d_date,sumws,sum,isEmpty] + CometProject [item_sk,d_date,sumws,sumws] + CometBroadcastHashJoin [item_sk,d_date,sumws,rk,item_sk,sumws,rk] + CometProject [item_sk,d_date,sumws,rk] + CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometExchange [ws_item_sk] #4 + CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [item_sk,sumws,rk] #8 + CometProject [item_sk,sumws] [item_sk,sumws,rk] + CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometExchange [item_sk,d_date] #9 + CometHashAggregate [sumss,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumss)] + CometExchange [item_sk,d_date,sumss] #10 + CometHashAggregate [sumss] [item_sk,d_date,sumss,sum,isEmpty] + CometProject [item_sk,d_date,sumss,sumss] + CometBroadcastHashJoin [item_sk,d_date,sumss,rk,item_sk,sumss,rk] + CometProject [item_sk,d_date,sumss,rk] + CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 + CometBroadcastExchange [item_sk,sumss,rk] #13 + CometProject [item_sk,sumss] [item_sk,sumss,rk] + CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + CometBroadcastExchange [item_sk,web_sales,store_sales,rk] #14 + CometProject [item_sk,web_sales,store_sales] [item_sk,web_sales,store_sales,rk] + CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #2 - WholeStageCodegen (5) - HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumws] #3 - WholeStageCodegen (4) - HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumws,sumws] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - InputAdapter - BroadcastExchange #8 - WholeStageCodegen (3) - Project [item_sk,sumws,rk] - InputAdapter - Window [ws_item_sk,d_date] - WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometColumnarExchange [item_sk,d_date] #9 - WholeStageCodegen (10) - HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [item_sk,d_date,sumss] #10 - WholeStageCodegen (9) - HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] - Project [item_sk,d_date,sumss,sumss] - BroadcastHashJoin [item_sk,item_sk,rk,rk] - Project [item_sk,d_date,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 - InputAdapter - BroadcastExchange #13 - WholeStageCodegen (8) - Project [item_sk,sumss,rk] - InputAdapter - Window [ss_item_sk,d_date] - WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - InputAdapter - BroadcastExchange #14 - WholeStageCodegen (23) - Project [item_sk,web_sales,store_sales,rk] - InputAdapter - Window [item_sk,d_date] - WholeStageCodegen (22) - CometColumnarToRow - InputAdapter - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt index d1daee98c1..c5215aa2f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt @@ -1,52 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (28) + : : +- CometFilter (27) + : : +- CometWindowExec (26) + : : +- CometFilter (25) + : : +- CometWindowExec (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometWindowExec (33) + : +- CometSort (32) + : +- CometExchange (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + +- ReusedExchange (38) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -159,131 +153,107 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] +(24) CometWindowExec Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] +(25) CometFilter Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window +(26) CometWindowExec Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20], [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(28) Filter [codegen id : 7] +(27) CometFilter Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] +(28) CometProject Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -(30) ReusedExchange [Reuses operator id: 20] +(29) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -(31) CometHashAggregate +(30) CometHashAggregate Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -(32) CometExchange +(31) CometExchange Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(33) CometSort +(32) CometSort Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window +(33) CometWindowExec Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26], [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] +(34) CometProject Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] +Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26], [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -(37) BroadcastExchange +(35) CometBroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] +Right output [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)], Inner, BuildRight -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +(37) CometProject Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] +(38) ReusedExchange [Reuses operator id: 35] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +Right output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#30, i_brand#31, cc_name#32, (rn#34 - 1)], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None +(40) CometProject +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +Arguments: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#35, sum_sales#33 AS nsum#36] -(47) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +(41) CometTakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#20,sum_sales#17,psum#35,nsum#36]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], 100, 0, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] -(48) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +(42) CometColumnarToRow [codegen id : 1] +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(44) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(46) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt index 101b15d3ac..00de5f32a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt @@ -1,78 +1,76 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt index d655789fe6..7a51065baf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt @@ -1,71 +1,51 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,cc_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,cc_name] #8 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index d1daee98c1..c5215aa2f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -1,52 +1,46 @@ == Physical Plan == -TakeOrderedAndProject (48) -+- * Project (47) - +- * BroadcastHashJoin Inner BuildRight (46) - :- * Project (39) - : +- * BroadcastHashJoin Inner BuildRight (38) - : :- * Project (29) - : : +- * Filter (28) - : : +- Window (27) - : : +- * Filter (26) - : : +- Window (25) - : : +- * CometColumnarToRow (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- BroadcastExchange (37) - : +- * Project (36) - : +- Window (35) - : +- * CometColumnarToRow (34) - : +- CometSort (33) - : +- CometExchange (32) - : +- CometHashAggregate (31) - : +- ReusedExchange (30) - +- BroadcastExchange (45) - +- * Project (44) - +- Window (43) - +- * CometColumnarToRow (42) - +- CometSort (41) - +- ReusedExchange (40) +* CometColumnarToRow (42) ++- CometTakeOrderedAndProject (41) + +- CometProject (40) + +- CometBroadcastHashJoin (39) + :- CometProject (37) + : +- CometBroadcastHashJoin (36) + : :- CometProject (28) + : : +- CometFilter (27) + : : +- CometWindowExec (26) + : : +- CometFilter (25) + : : +- CometWindowExec (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- CometBroadcastExchange (35) + : +- CometProject (34) + : +- CometWindowExec (33) + : +- CometSort (32) + : +- CometExchange (31) + : +- CometHashAggregate (30) + : +- ReusedExchange (29) + +- ReusedExchange (38) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -159,131 +153,107 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] +(24) CometWindowExec Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) Window -Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] - -(26) Filter [codegen id : 2] +(25) CometFilter Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) Window +(26) CometWindowExec Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20], [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(28) Filter [codegen id : 7] +(27) CometFilter Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(29) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] +(28) CometProject Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -(30) ReusedExchange [Reuses operator id: 20] +(29) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -(31) CometHashAggregate +(30) CometHashAggregate Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -(32) CometExchange +(31) CometExchange Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(33) CometSort +(32) CometSort Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(34) CometColumnarToRow [codegen id : 3] -Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] - -(35) Window +(33) CometWindowExec Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26], [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(36) Project [codegen id : 4] -Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] +(34) CometProject Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] +Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26], [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -(37) BroadcastExchange +(35) CometBroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] +Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -(38) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] -Join type: Inner -Join condition: None +(36) CometBroadcastHashJoin +Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] +Right output [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)], Inner, BuildRight -(39) Project [codegen id : 7] -Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +(37) CometProject Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -(40) ReusedExchange [Reuses operator id: 32] -Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(41) CometSort -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(42) CometColumnarToRow [codegen id : 5] -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] - -(43) Window -Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] -Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] - -(44) Project [codegen id : 6] -Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] -Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] +(38) ReusedExchange [Reuses operator id: 35] +Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -(45) BroadcastExchange -Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] +(39) CometBroadcastHashJoin +Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] +Right output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#30, i_brand#31, cc_name#32, (rn#34 - 1)], Inner, BuildRight -(46) BroadcastHashJoin [codegen id : 7] -Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] -Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] -Join type: Inner -Join condition: None +(40) CometProject +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +Arguments: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#35, sum_sales#33 AS nsum#36] -(47) Project [codegen id : 7] -Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +(41) CometTakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#20,sum_sales#17,psum#35,nsum#36]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], 100, 0, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] -(48) TakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] -Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +(42) CometColumnarToRow [codegen id : 1] +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (52) -+- * CometColumnarToRow (51) - +- CometFilter (50) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometFilter (44) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(50) CometFilter +(44) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(51) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(52) BroadcastExchange +(46) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt index 101b15d3ac..00de5f32a1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt @@ -1,78 +1,76 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- BroadcastHashJoin - :- Project - : +- BroadcastHashJoin - : :- Project - : : +- Filter - : : +- Window - : : +- Filter - : : +- Window - : : +- CometColumnarToRow - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- BroadcastExchange - : +- Project - : +- Window - : +- CometColumnarToRow - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- BroadcastExchange - +- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometFilter + : : +- CometWindowExec + : : +- CometFilter + : : +- CometWindowExec + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- CometBroadcastExchange + : +- CometProject + : +- CometWindowExec + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- CometBroadcastExchange + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +100,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index d655789fe6..7a51065baf 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -1,71 +1,51 @@ -TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] - WholeStageCodegen (7) - Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] - Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - Filter [avg_monthly_sales,sum_sales] - InputAdapter - Window [_w0,i_category,i_brand,cc_name,d_year] - WholeStageCodegen (2) - Filter [d_year] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (4) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - InputAdapter - BroadcastExchange #9 - WholeStageCodegen (6) - Project [i_category,i_brand,cc_name,sum_sales,rn] - InputAdapter - Window [d_year,d_moy,i_category,i_brand,cc_name] - WholeStageCodegen (5) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometProject [sum_sales,sum_sales] [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] + CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] + CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,cc_name,sum_sales,rn] + CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,cc_name] #8 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt index ac04f86f96..26a54ba072 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * Filter (69) - +- Window (68) - +- * CometColumnarToRow (67) +* CometColumnarToRow (70) ++- CometTakeOrderedAndProject (69) + +- CometFilter (68) + +- CometWindowExec (67) +- CometSort (66) +- CometExchange (65) +- CometUnion (64) @@ -376,20 +376,20 @@ Arguments: hashpartitioning(i_category#99, 5), ENSURE_REQUIREMENTS, CometNativeS Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] Arguments: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107], [i_category#99 ASC NULLS FIRST, sumsales#107 DESC NULLS LAST] -(67) CometColumnarToRow [codegen id : 1] +(67) CometWindowExec Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] +Arguments: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152], [rank(sumsales#107) windowspecdefinition(i_category#99, sumsales#107 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#152], [i_category#99], [sumsales#107 DESC NULLS LAST] -(68) Window -Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Arguments: [rank(sumsales#107) windowspecdefinition(i_category#99, sumsales#107 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#152], [i_category#99], [sumsales#107 DESC NULLS LAST] - -(69) Filter [codegen id : 2] +(68) CometFilter Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] Condition : (rk#152 <= 100) -(70) TakeOrderedAndProject +(69) CometTakeOrderedAndProject +Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#99 ASC NULLS FIRST,i_class#100 ASC NULLS FIRST,i_brand#101 ASC NULLS FIRST,i_product_name#102 ASC NULLS FIRST,d_year#103 ASC NULLS FIRST,d_qoy#104 ASC NULLS FIRST,d_moy#105 ASC NULLS FIRST,s_store_id#106 ASC NULLS FIRST,sumsales#107 ASC NULLS FIRST,rk#152 ASC NULLS FIRST], output=[i_category#99,i_class#100,i_brand#101,i_product_name#102,d_year#103,d_qoy#104,d_moy#105,s_store_id#106,sumsales#107,rk#152]), [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152], 100, 0, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_brand#101 ASC NULLS FIRST, i_product_name#102 ASC NULLS FIRST, d_year#103 ASC NULLS FIRST, d_qoy#104 ASC NULLS FIRST, d_moy#105 ASC NULLS FIRST, s_store_id#106 ASC NULLS FIRST, sumsales#107 ASC NULLS FIRST, rk#152 ASC NULLS FIRST], [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] + +(70) CometColumnarToRow [codegen id : 1] Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] -Arguments: 100, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_brand#101 ASC NULLS FIRST, i_product_name#102 ASC NULLS FIRST, d_year#103 ASC NULLS FIRST, d_qoy#104 ASC NULLS FIRST, d_moy#105 ASC NULLS FIRST, s_store_id#106 ASC NULLS FIRST, sumsales#107 ASC NULLS FIRST, rk#152 ASC NULLS FIRST], [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt index 7b7ba7df09..2eaf297aaa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometUnion @@ -291,4 +291,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 264 out of 282 eligible operators (93%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt index 5ca258aeca..cd140526f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt @@ -1,82 +1,80 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (2) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometFilter [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometWindowExec [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometExchange [i_category] #1 + CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] + CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] + CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] + CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name] #10 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] + CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand] #11 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] + CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class] #12 + CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] + CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category] #13 + CometHashAggregate [sumsales] [i_category,sum,isEmpty] + CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange #14 + CometHashAggregate [sumsales] [sum,isEmpty] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index ac04f86f96..26a54ba072 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (70) -+- * Filter (69) - +- Window (68) - +- * CometColumnarToRow (67) +* CometColumnarToRow (70) ++- CometTakeOrderedAndProject (69) + +- CometFilter (68) + +- CometWindowExec (67) +- CometSort (66) +- CometExchange (65) +- CometUnion (64) @@ -376,20 +376,20 @@ Arguments: hashpartitioning(i_category#99, 5), ENSURE_REQUIREMENTS, CometNativeS Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] Arguments: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107], [i_category#99 ASC NULLS FIRST, sumsales#107 DESC NULLS LAST] -(67) CometColumnarToRow [codegen id : 1] +(67) CometWindowExec Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] +Arguments: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152], [rank(sumsales#107) windowspecdefinition(i_category#99, sumsales#107 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#152], [i_category#99], [sumsales#107 DESC NULLS LAST] -(68) Window -Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Arguments: [rank(sumsales#107) windowspecdefinition(i_category#99, sumsales#107 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#152], [i_category#99], [sumsales#107 DESC NULLS LAST] - -(69) Filter [codegen id : 2] +(68) CometFilter Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] Condition : (rk#152 <= 100) -(70) TakeOrderedAndProject +(69) CometTakeOrderedAndProject +Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#99 ASC NULLS FIRST,i_class#100 ASC NULLS FIRST,i_brand#101 ASC NULLS FIRST,i_product_name#102 ASC NULLS FIRST,d_year#103 ASC NULLS FIRST,d_qoy#104 ASC NULLS FIRST,d_moy#105 ASC NULLS FIRST,s_store_id#106 ASC NULLS FIRST,sumsales#107 ASC NULLS FIRST,rk#152 ASC NULLS FIRST], output=[i_category#99,i_class#100,i_brand#101,i_product_name#102,d_year#103,d_qoy#104,d_moy#105,s_store_id#106,sumsales#107,rk#152]), [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152], 100, 0, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_brand#101 ASC NULLS FIRST, i_product_name#102 ASC NULLS FIRST, d_year#103 ASC NULLS FIRST, d_qoy#104 ASC NULLS FIRST, d_moy#105 ASC NULLS FIRST, s_store_id#106 ASC NULLS FIRST, sumsales#107 ASC NULLS FIRST, rk#152 ASC NULLS FIRST], [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] + +(70) CometColumnarToRow [codegen id : 1] Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] -Arguments: 100, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_brand#101 ASC NULLS FIRST, i_product_name#102 ASC NULLS FIRST, d_year#103 ASC NULLS FIRST, d_qoy#104 ASC NULLS FIRST, d_moy#105 ASC NULLS FIRST, s_store_id#106 ASC NULLS FIRST, sumsales#107 ASC NULLS FIRST, rk#152 ASC NULLS FIRST], [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt index 7b7ba7df09..2eaf297aaa 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometFilter + +- CometWindowExec +- CometSort +- CometExchange +- CometUnion @@ -291,4 +291,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 264 out of 282 eligible operators (93%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index 5ca258aeca..cd140526f4 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -1,82 +1,80 @@ -TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - WholeStageCodegen (2) - Filter [rk] - InputAdapter - Window [sumsales,i_category] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometFilter [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometWindowExec [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometExchange [i_category] #1 + CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] + CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] + CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] + CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name] #10 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] + CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand] #11 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] + CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class] #12 + CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] + CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category] #13 + CometHashAggregate [sumsales] [i_category,sum,isEmpty] + CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange #14 + CometHashAggregate [sumsales] [sum,isEmpty] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt index 02cdbb3c66..ae6e2e9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt @@ -1,69 +1,61 @@ == Physical Plan == -TakeOrderedAndProject (65) -+- * Project (64) - +- Window (63) - +- * CometColumnarToRow (62) - +- CometSort (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- Union (56) - :- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometColumnarExchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - : +- BroadcastExchange (32) - : +- * Project (31) - : +- * Filter (30) - : +- Window (29) - : +- * CometColumnarToRow (28) - : +- CometSort (27) - : +- CometHashAggregate (26) - : +- CometExchange (25) - : +- CometHashAggregate (24) - : +- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- ReusedExchange (21) - :- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- CometColumnarExchange (46) - : +- * HashAggregate (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- ReusedExchange (42) - +- * HashAggregate (55) - +- * CometColumnarToRow (54) - +- CometColumnarExchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * CometColumnarToRow (50) - +- ReusedExchange (49) +* CometColumnarToRow (57) ++- CometTakeOrderedAndProject (56) + +- CometProject (55) + +- CometWindowExec (54) + +- CometSort (53) + +- CometExchange (52) + +- CometHashAggregate (51) + +- CometExchange (50) + +- CometHashAggregate (49) + +- CometUnion (48) + :- CometHashAggregate (37) + : +- CometExchange (36) + : +- CometHashAggregate (35) + : +- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (32) + : +- CometProject (31) + : +- CometBroadcastHashJoin (30) + : :- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (29) + : +- CometProject (28) + : +- CometFilter (27) + : +- CometWindowExec (26) + : +- CometSort (25) + : +- CometHashAggregate (24) + : +- CometExchange (23) + : +- CometHashAggregate (22) + : +- CometProject (21) + : +- CometBroadcastHashJoin (20) + : :- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometFilter (12) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) + : : +- CometBroadcastExchange (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) + : +- ReusedExchange (19) + :- CometHashAggregate (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometHashAggregate (39) + : +- ReusedExchange (38) + +- CometHashAggregate (47) + +- CometExchange (46) + +- CometHashAggregate (45) + +- CometHashAggregate (44) + +- ReusedExchange (43) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -106,24 +98,18 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] @@ -131,273 +117,238 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Condition : isnotnull(ss_store_sk#10) -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [2]: [s_store_sk#14, s_state#15] Condition : isnotnull(s_store_sk#14) -(17) CometProject +(15) CometProject Input [2]: [s_store_sk#14, s_state#15] Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] -(18) CometBroadcastExchange +(16) CometBroadcastExchange Input [2]: [s_store_sk#14, s_state#16] Arguments: [s_store_sk#14, s_state#16] -(19) CometBroadcastHashJoin +(17) CometBroadcastHashJoin Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Right output [2]: [s_store_sk#14, s_state#16] Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight -(20) CometProject +(18) CometProject Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -(21) ReusedExchange [Reuses operator id: 6] +(19) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#17] -(22) CometBroadcastHashJoin +(20) CometBroadcastHashJoin Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] Right output [1]: [d_date_sk#17] Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight -(23) CometProject +(21) CometProject Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] -(24) CometHashAggregate +(22) CometHashAggregate Input [2]: [ss_net_profit#11, s_state#16] Keys [1]: [s_state#16] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] -(25) CometExchange +(23) CometExchange Input [2]: [s_state#16, sum#18] Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(26) CometHashAggregate +(24) CometHashAggregate Input [2]: [s_state#16, sum#18] Keys [1]: [s_state#16] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -(27) CometSort +(25) CometSort Input [3]: [s_state#16, _w0#19, s_state#16] Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(28) CometColumnarToRow [codegen id : 1] +(26) CometWindowExec Input [3]: [s_state#16, _w0#19, s_state#16] +Arguments: [s_state#16, _w0#19, s_state#16, ranking#20], [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] -(29) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(30) Filter [codegen id : 2] +(27) CometFilter Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] Condition : (ranking#20 <= 5) -(31) Project [codegen id : 2] -Output [1]: [s_state#16] +(28) CometProject Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] +Arguments: [s_state#16], [s_state#16] -(32) BroadcastExchange +(29) CometBroadcastExchange Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] +Arguments: [s_state#16] -(33) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None +(30) CometBroadcastHashJoin +Left output [3]: [s_store_sk#7, s_county#8, s_state#9] +Right output [1]: [s_state#16] +Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)], [s_state#16], LeftSemi, BuildRight -(34) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] +(31) CometProject Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Arguments: [s_store_sk#7, s_county#8, s_state#21], [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -(35) BroadcastExchange +(32) CometBroadcastExchange Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [s_store_sk#7, s_county#8, s_state#21] -(36) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None +(33) CometBroadcastHashJoin +Left output [2]: [ss_store_sk#1, ss_net_profit#2] +Right output [3]: [s_store_sk#7, s_county#8, s_state#21] +Arguments: [ss_store_sk#1], [s_store_sk#7], Inner, BuildRight -(37) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#21] +(34) CometProject Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] +Arguments: [ss_net_profit#2, s_county#8, s_state#21], [ss_net_profit#2, s_county#8, s_state#21] -(38) HashAggregate [codegen id : 4] +(35) CometHashAggregate Input [3]: [ss_net_profit#2, s_county#8, s_state#21] Keys [2]: [s_state#21, s_county#8] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#8, sum#23] -(39) CometColumnarExchange -Input [3]: [s_state#21, s_county#8, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(36) CometExchange +Input [3]: [s_state#21, s_county#8, sum#22] +Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(40) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] - -(41) HashAggregate [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] +(37) CometHashAggregate +Input [3]: [s_state#21, s_county#8, sum#22] Keys [2]: [s_state#21, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(42) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#21, s_county#31, sum#32] -(43) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] +(38) ReusedExchange [Reuses operator id: 36] +Output [3]: [s_state#21, s_county#23, sum#24] -(44) HashAggregate [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] -Keys [2]: [s_state#21, s_county#31] -Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] +(39) CometHashAggregate +Input [3]: [s_state#21, s_county#23, sum#24] +Keys [2]: [s_state#21, s_county#23] +Functions [1]: [sum(UnscaledValue(ss_net_profit#25))] -(45) HashAggregate [codegen id : 10] -Input [2]: [total_sum#34, s_state#21] +(40) CometHashAggregate +Input [2]: [total_sum#26, s_state#21] Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [3]: [s_state#21, sum#37, isEmpty#38] +Functions [1]: [partial_sum(total_sum#26)] -(46) CometColumnarExchange -Input [3]: [s_state#21, sum#37, isEmpty#38] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(41) CometExchange +Input [3]: [s_state#21, sum#27, isEmpty#28] +Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(47) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] - -(48) HashAggregate [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] +(42) CometHashAggregate +Input [3]: [s_state#21, sum#27, isEmpty#28] Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#39] -Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] - -(49) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#21, s_county#45, sum#46] +Functions [1]: [sum(total_sum#26)] -(50) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] +(43) ReusedExchange [Reuses operator id: 36] +Output [3]: [s_state#21, s_county#29, sum#30] -(51) HashAggregate [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] -Keys [2]: [s_state#21, s_county#45] -Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#34] +(44) CometHashAggregate +Input [3]: [s_state#21, s_county#29, sum#30] +Keys [2]: [s_state#21, s_county#29] +Functions [1]: [sum(UnscaledValue(ss_net_profit#31))] -(52) HashAggregate [codegen id : 16] -Input [1]: [total_sum#34] +(45) CometHashAggregate +Input [1]: [total_sum#26] Keys: [] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [2]: [sum#50, isEmpty#51] - -(53) CometColumnarExchange -Input [2]: [sum#50, isEmpty#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Functions [1]: [partial_sum(total_sum#26)] -(54) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#50, isEmpty#51] +(46) CometExchange +Input [2]: [sum#32, isEmpty#33] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(55) HashAggregate [codegen id : 17] -Input [2]: [sum#50, isEmpty#51] +(47) CometHashAggregate +Input [2]: [sum#32, isEmpty#33] Keys: [] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#52] -Results [6]: [sum(total_sum#34)#52 AS total_sum#53, null AS s_state#54, null AS s_county#55, 1 AS g_state#56, 1 AS g_county#57, 2 AS lochierarchy#58] +Functions [1]: [sum(total_sum#26)] -(56) Union +(48) CometUnion +Child 0 Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +Child 1 Input [6]: [total_sum#40, s_state#21, s_county#41, g_state#42, g_county#43, lochierarchy#44] +Child 2 Input [6]: [total_sum#45, s_state#46, s_county#47, g_state#48, g_county#49, lochierarchy#50] -(57) HashAggregate [codegen id : 18] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +(49) CometHashAggregate +Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +Keys [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -(58) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(50) CometExchange +Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +Arguments: hashpartitioning(total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(59) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +(51) CometHashAggregate +Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +Keys [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] Functions: [] -(60) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: hashpartitioning(lochierarchy#30, _w0#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(52) CometExchange +Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] +Arguments: hashpartitioning(lochierarchy#39, _w0#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(61) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59], [lochierarchy#30 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] +(53) CometSort +Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] +Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51], [lochierarchy#39 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#34 DESC NULLS LAST] -(62) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] +(54) CometWindowExec +Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] +Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51, rank_within_parent#52], [rank(total_sum#34) windowspecdefinition(lochierarchy#39, _w0#51, total_sum#34 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#39, _w0#51], [total_sum#34 DESC NULLS LAST] -(63) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#59, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#30, _w0#59], [total_sum#25 DESC NULLS LAST] +(55) CometProject +Input [6]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51, rank_within_parent#52] +Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52], [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] -(64) Project [codegen id : 20] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59, rank_within_parent#60] +(56) CometTakeOrderedAndProject +Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#39 DESC NULLS LAST,CASE WHEN (lochierarchy#39 = 0) THEN s_state#35 END ASC NULLS FIRST,rank_within_parent#52 ASC NULLS FIRST], output=[total_sum#34,s_state#35,s_county#36,lochierarchy#39,rank_within_parent#52]), [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52], 100, 0, [lochierarchy#39 DESC NULLS LAST, CASE WHEN (lochierarchy#39 = 0) THEN s_state#35 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] -(65) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] +(57) CometColumnarToRow [codegen id : 1] +Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) +BroadcastExchange (62) ++- * CometColumnarToRow (61) + +- CometProject (60) + +- CometFilter (59) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(67) CometFilter +(59) CometFilter Input [2]: [d_date_sk#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(68) CometProject +(60) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(69) CometColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(70) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt index 59af759973..5b14945482 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt @@ -1,173 +1,159 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometHashAggregate + : +- CometExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 153 eligible operators (66%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 144 out of 153 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt index 8de8b7e637..91be7bc024 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt @@ -1,100 +1,68 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,rank_within_parent] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + CometHashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + CometUnion [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + CometHashAggregate [sum] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(UnscaledValue(ss_net_profit)),s_state,s_county] + CometExchange [s_state,s_county] #3 + CometHashAggregate [ss_net_profit] [s_state,s_county,sum] + CometProject [ss_net_profit,s_county,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_county,s_state] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [s_store_sk,s_county,s_state] #6 + CometProject [s_state] [s_store_sk,s_county,s_state] + CometBroadcastHashJoin [s_store_sk,s_county,s_state,s_state] + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + CometBroadcastExchange [s_state] #7 + CometProject [s_state] + CometFilter [s_state,_w0,ranking] + CometWindowExec [s_state,_w0,ranking] + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] + CometExchange [s_state] #8 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #9 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + CometHashAggregate [sum,isEmpty] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(total_sum)] + CometExchange [s_state] #10 + CometHashAggregate [total_sum] [s_state,sum,isEmpty] + CometHashAggregate [s_county,sum] [total_sum,s_state,sum(UnscaledValue(ss_net_profit))] + ReusedExchange [s_state,s_county,sum] #3 + CometHashAggregate [sum,isEmpty] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(total_sum)] + CometExchange #11 + CometHashAggregate [total_sum] [sum,isEmpty] + CometHashAggregate [s_state,s_county,sum] [total_sum,sum(UnscaledValue(ss_net_profit))] + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index 02cdbb3c66..ae6e2e9bd2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -1,69 +1,61 @@ == Physical Plan == -TakeOrderedAndProject (65) -+- * Project (64) - +- Window (63) - +- * CometColumnarToRow (62) - +- CometSort (61) - +- CometExchange (60) - +- CometHashAggregate (59) - +- CometColumnarExchange (58) - +- * HashAggregate (57) - +- Union (56) - :- * HashAggregate (41) - : +- * CometColumnarToRow (40) - : +- CometColumnarExchange (39) - : +- * HashAggregate (38) - : +- * Project (37) - : +- * BroadcastHashJoin Inner BuildRight (36) - : :- * CometColumnarToRow (9) - : : +- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- BroadcastExchange (35) - : +- * Project (34) - : +- * BroadcastHashJoin LeftSemi BuildRight (33) - : :- * CometColumnarToRow (12) - : : +- CometFilter (11) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) - : +- BroadcastExchange (32) - : +- * Project (31) - : +- * Filter (30) - : +- Window (29) - : +- * CometColumnarToRow (28) - : +- CometSort (27) - : +- CometHashAggregate (26) - : +- CometExchange (25) - : +- CometHashAggregate (24) - : +- CometProject (23) - : +- CometBroadcastHashJoin (22) - : :- CometProject (20) - : : +- CometBroadcastHashJoin (19) - : : :- CometFilter (14) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) - : : +- CometBroadcastExchange (18) - : : +- CometProject (17) - : : +- CometFilter (16) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) - : +- ReusedExchange (21) - :- * HashAggregate (48) - : +- * CometColumnarToRow (47) - : +- CometColumnarExchange (46) - : +- * HashAggregate (45) - : +- * HashAggregate (44) - : +- * CometColumnarToRow (43) - : +- ReusedExchange (42) - +- * HashAggregate (55) - +- * CometColumnarToRow (54) - +- CometColumnarExchange (53) - +- * HashAggregate (52) - +- * HashAggregate (51) - +- * CometColumnarToRow (50) - +- ReusedExchange (49) +* CometColumnarToRow (57) ++- CometTakeOrderedAndProject (56) + +- CometProject (55) + +- CometWindowExec (54) + +- CometSort (53) + +- CometExchange (52) + +- CometHashAggregate (51) + +- CometExchange (50) + +- CometHashAggregate (49) + +- CometUnion (48) + :- CometHashAggregate (37) + : +- CometExchange (36) + : +- CometHashAggregate (35) + : +- CometProject (34) + : +- CometBroadcastHashJoin (33) + : :- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : +- CometBroadcastExchange (32) + : +- CometProject (31) + : +- CometBroadcastHashJoin (30) + : :- CometFilter (10) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) + : +- CometBroadcastExchange (29) + : +- CometProject (28) + : +- CometFilter (27) + : +- CometWindowExec (26) + : +- CometSort (25) + : +- CometHashAggregate (24) + : +- CometExchange (23) + : +- CometHashAggregate (22) + : +- CometProject (21) + : +- CometBroadcastHashJoin (20) + : :- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometFilter (12) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) + : : +- CometBroadcastExchange (16) + : : +- CometProject (15) + : : +- CometFilter (14) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) + : +- ReusedExchange (19) + :- CometHashAggregate (42) + : +- CometExchange (41) + : +- CometHashAggregate (40) + : +- CometHashAggregate (39) + : +- ReusedExchange (38) + +- CometHashAggregate (47) + +- CometExchange (46) + +- CometHashAggregate (45) + +- CometHashAggregate (44) + +- ReusedExchange (43) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -106,24 +98,18 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) CometColumnarToRow [codegen id : 4] -Input [2]: [ss_store_sk#1, ss_net_profit#2] - -(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(11) CometFilter +(10) CometFilter Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(12) CometColumnarToRow [codegen id : 3] -Input [3]: [s_store_sk#7, s_county#8, s_state#9] - -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] @@ -131,273 +117,238 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(14) CometFilter +(12) CometFilter Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Condition : isnotnull(ss_store_sk#10) -(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(16) CometFilter +(14) CometFilter Input [2]: [s_store_sk#14, s_state#15] Condition : isnotnull(s_store_sk#14) -(17) CometProject +(15) CometProject Input [2]: [s_store_sk#14, s_state#15] Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] -(18) CometBroadcastExchange +(16) CometBroadcastExchange Input [2]: [s_store_sk#14, s_state#16] Arguments: [s_store_sk#14, s_state#16] -(19) CometBroadcastHashJoin +(17) CometBroadcastHashJoin Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Right output [2]: [s_store_sk#14, s_state#16] Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight -(20) CometProject +(18) CometProject Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -(21) ReusedExchange [Reuses operator id: 6] +(19) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#17] -(22) CometBroadcastHashJoin +(20) CometBroadcastHashJoin Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] Right output [1]: [d_date_sk#17] Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight -(23) CometProject +(21) CometProject Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] -(24) CometHashAggregate +(22) CometHashAggregate Input [2]: [ss_net_profit#11, s_state#16] Keys [1]: [s_state#16] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] -(25) CometExchange +(23) CometExchange Input [2]: [s_state#16, sum#18] Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(26) CometHashAggregate +(24) CometHashAggregate Input [2]: [s_state#16, sum#18] Keys [1]: [s_state#16] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -(27) CometSort +(25) CometSort Input [3]: [s_state#16, _w0#19, s_state#16] Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(28) CometColumnarToRow [codegen id : 1] +(26) CometWindowExec Input [3]: [s_state#16, _w0#19, s_state#16] +Arguments: [s_state#16, _w0#19, s_state#16, ranking#20], [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] -(29) Window -Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] - -(30) Filter [codegen id : 2] +(27) CometFilter Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] Condition : (ranking#20 <= 5) -(31) Project [codegen id : 2] -Output [1]: [s_state#16] +(28) CometProject Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] +Arguments: [s_state#16], [s_state#16] -(32) BroadcastExchange +(29) CometBroadcastExchange Input [1]: [s_state#16] -Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] +Arguments: [s_state#16] -(33) BroadcastHashJoin [codegen id : 3] -Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] -Right keys [1]: [s_state#16] -Join type: LeftSemi -Join condition: None +(30) CometBroadcastHashJoin +Left output [3]: [s_store_sk#7, s_county#8, s_state#9] +Right output [1]: [s_state#16] +Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)], [s_state#16], LeftSemi, BuildRight -(34) Project [codegen id : 3] -Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] +(31) CometProject Input [3]: [s_store_sk#7, s_county#8, s_state#9] +Arguments: [s_store_sk#7, s_county#8, s_state#21], [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -(35) BroadcastExchange +(32) CometBroadcastExchange Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] +Arguments: [s_store_sk#7, s_county#8, s_state#21] -(36) BroadcastHashJoin [codegen id : 4] -Left keys [1]: [ss_store_sk#1] -Right keys [1]: [s_store_sk#7] -Join type: Inner -Join condition: None +(33) CometBroadcastHashJoin +Left output [2]: [ss_store_sk#1, ss_net_profit#2] +Right output [3]: [s_store_sk#7, s_county#8, s_state#21] +Arguments: [ss_store_sk#1], [s_store_sk#7], Inner, BuildRight -(37) Project [codegen id : 4] -Output [3]: [ss_net_profit#2, s_county#8, s_state#21] +(34) CometProject Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] +Arguments: [ss_net_profit#2, s_county#8, s_state#21], [ss_net_profit#2, s_county#8, s_state#21] -(38) HashAggregate [codegen id : 4] +(35) CometHashAggregate Input [3]: [ss_net_profit#2, s_county#8, s_state#21] Keys [2]: [s_state#21, s_county#8] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum#22] -Results [3]: [s_state#21, s_county#8, sum#23] -(39) CometColumnarExchange -Input [3]: [s_state#21, s_county#8, sum#23] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] +(36) CometExchange +Input [3]: [s_state#21, s_county#8, sum#22] +Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(40) CometColumnarToRow [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] - -(41) HashAggregate [codegen id : 5] -Input [3]: [s_state#21, s_county#8, sum#23] +(37) CometHashAggregate +Input [3]: [s_state#21, s_county#8, sum#22] Keys [2]: [s_state#21, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] -Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] - -(42) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#21, s_county#31, sum#32] -(43) CometColumnarToRow [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] +(38) ReusedExchange [Reuses operator id: 36] +Output [3]: [s_state#21, s_county#23, sum#24] -(44) HashAggregate [codegen id : 10] -Input [3]: [s_state#21, s_county#31, sum#32] -Keys [2]: [s_state#21, s_county#31] -Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] -Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] +(39) CometHashAggregate +Input [3]: [s_state#21, s_county#23, sum#24] +Keys [2]: [s_state#21, s_county#23] +Functions [1]: [sum(UnscaledValue(ss_net_profit#25))] -(45) HashAggregate [codegen id : 10] -Input [2]: [total_sum#34, s_state#21] +(40) CometHashAggregate +Input [2]: [total_sum#26, s_state#21] Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#35, isEmpty#36] -Results [3]: [s_state#21, sum#37, isEmpty#38] +Functions [1]: [partial_sum(total_sum#26)] -(46) CometColumnarExchange -Input [3]: [s_state#21, sum#37, isEmpty#38] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] +(41) CometExchange +Input [3]: [s_state#21, sum#27, isEmpty#28] +Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(47) CometColumnarToRow [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] - -(48) HashAggregate [codegen id : 11] -Input [3]: [s_state#21, sum#37, isEmpty#38] +(42) CometHashAggregate +Input [3]: [s_state#21, sum#27, isEmpty#28] Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#39] -Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] - -(49) ReusedExchange [Reuses operator id: 39] -Output [3]: [s_state#21, s_county#45, sum#46] +Functions [1]: [sum(total_sum#26)] -(50) CometColumnarToRow [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] +(43) ReusedExchange [Reuses operator id: 36] +Output [3]: [s_state#21, s_county#29, sum#30] -(51) HashAggregate [codegen id : 16] -Input [3]: [s_state#21, s_county#45, sum#46] -Keys [2]: [s_state#21, s_county#45] -Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] -Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#34] +(44) CometHashAggregate +Input [3]: [s_state#21, s_county#29, sum#30] +Keys [2]: [s_state#21, s_county#29] +Functions [1]: [sum(UnscaledValue(ss_net_profit#31))] -(52) HashAggregate [codegen id : 16] -Input [1]: [total_sum#34] +(45) CometHashAggregate +Input [1]: [total_sum#26] Keys: [] -Functions [1]: [partial_sum(total_sum#34)] -Aggregate Attributes [2]: [sum#48, isEmpty#49] -Results [2]: [sum#50, isEmpty#51] - -(53) CometColumnarExchange -Input [2]: [sum#50, isEmpty#51] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Functions [1]: [partial_sum(total_sum#26)] -(54) CometColumnarToRow [codegen id : 17] -Input [2]: [sum#50, isEmpty#51] +(46) CometExchange +Input [2]: [sum#32, isEmpty#33] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(55) HashAggregate [codegen id : 17] -Input [2]: [sum#50, isEmpty#51] +(47) CometHashAggregate +Input [2]: [sum#32, isEmpty#33] Keys: [] -Functions [1]: [sum(total_sum#34)] -Aggregate Attributes [1]: [sum(total_sum#34)#52] -Results [6]: [sum(total_sum#34)#52 AS total_sum#53, null AS s_state#54, null AS s_county#55, 1 AS g_state#56, 1 AS g_county#57, 2 AS lochierarchy#58] +Functions [1]: [sum(total_sum#26)] -(56) Union +(48) CometUnion +Child 0 Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +Child 1 Input [6]: [total_sum#40, s_state#21, s_county#41, g_state#42, g_county#43, lochierarchy#44] +Child 2 Input [6]: [total_sum#45, s_state#46, s_county#47, g_state#48, g_county#49, lochierarchy#50] -(57) HashAggregate [codegen id : 18] -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +(49) CometHashAggregate +Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +Keys [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] Functions: [] -Aggregate Attributes: [] -Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -(58) CometColumnarExchange -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(50) CometExchange +Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +Arguments: hashpartitioning(total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(59) CometHashAggregate -Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +(51) CometHashAggregate +Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +Keys [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] Functions: [] -(60) CometExchange -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: hashpartitioning(lochierarchy#30, _w0#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(52) CometExchange +Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] +Arguments: hashpartitioning(lochierarchy#39, _w0#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(61) CometSort -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59], [lochierarchy#30 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] +(53) CometSort +Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] +Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51], [lochierarchy#39 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#34 DESC NULLS LAST] -(62) CometColumnarToRow [codegen id : 19] -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] +(54) CometWindowExec +Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] +Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51, rank_within_parent#52], [rank(total_sum#34) windowspecdefinition(lochierarchy#39, _w0#51, total_sum#34 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#39, _w0#51], [total_sum#34 DESC NULLS LAST] -(63) Window -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#59, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#30, _w0#59], [total_sum#25 DESC NULLS LAST] +(55) CometProject +Input [6]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51, rank_within_parent#52] +Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52], [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] -(64) Project [codegen id : 20] -Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] -Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59, rank_within_parent#60] +(56) CometTakeOrderedAndProject +Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#39 DESC NULLS LAST,CASE WHEN (lochierarchy#39 = 0) THEN s_state#35 END ASC NULLS FIRST,rank_within_parent#52 ASC NULLS FIRST], output=[total_sum#34,s_state#35,s_county#36,lochierarchy#39,rank_within_parent#52]), [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52], 100, 0, [lochierarchy#39 DESC NULLS LAST, CASE WHEN (lochierarchy#39 = 0) THEN s_state#35 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] -(65) TakeOrderedAndProject -Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] -Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] +(57) CometColumnarToRow [codegen id : 1] +Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (70) -+- * CometColumnarToRow (69) - +- CometProject (68) - +- CometFilter (67) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) +BroadcastExchange (62) ++- * CometColumnarToRow (61) + +- CometProject (60) + +- CometFilter (59) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) -(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(67) CometFilter +(59) CometFilter Input [2]: [d_date_sk#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(68) CometProject +(60) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(69) CometColumnarToRow [codegen id : 1] +(61) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(70) BroadcastExchange +(62) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] -Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt index 59af759973..5b14945482 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt @@ -1,173 +1,159 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate - +- CometColumnarExchange - +- HashAggregate - +- Union - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometUnion + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometHashAggregate + : +- CometExchange : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- HashAggregate - : +- CometColumnarToRow - : +- CometColumnarExchange - : +- HashAggregate - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- BroadcastExchange - : +- Project - : +- BroadcastHashJoin - : :- CometColumnarToRow - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- BroadcastExchange - : +- Project - : +- Filter - : +- Window [COMET: Partitioning and sorting specifications must be the same.] - : +- CometColumnarToRow - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometWindowExec + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- HashAggregate - +- CometColumnarToRow - +- CometColumnarExchange - +- HashAggregate - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- BroadcastExchange - +- Project - +- BroadcastHashJoin - :- CometColumnarToRow - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- BroadcastExchange - +- Project - +- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] - +- CometColumnarToRow - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastExchange + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometWindowExec + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 102 out of 153 eligible operators (66%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 144 out of 153 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 8de8b7e637..91be7bc024 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -1,100 +1,68 @@ -TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] - WholeStageCodegen (20) - Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,rank_within_parent] + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + CometHashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + CometUnion [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + CometHashAggregate [sum] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(UnscaledValue(ss_net_profit)),s_state,s_county] + CometExchange [s_state,s_county] #3 + CometHashAggregate [ss_net_profit] [s_state,s_county,sum] + CometProject [ss_net_profit,s_county,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_county,s_state] + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) CometColumnarToRow InputAdapter - ReusedExchange [s_state,s_county,sum] #3 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [s_store_sk,s_county,s_state] #6 + CometProject [s_state] [s_store_sk,s_county,s_state] + CometBroadcastHashJoin [s_store_sk,s_county,s_state,s_state] + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + CometBroadcastExchange [s_state] #7 + CometProject [s_state] + CometFilter [s_state,_w0,ranking] + CometWindowExec [s_state,_w0,ranking] + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] + CometExchange [s_state] #8 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #9 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + CometHashAggregate [sum,isEmpty] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(total_sum)] + CometExchange [s_state] #10 + CometHashAggregate [total_sum] [s_state,sum,isEmpty] + CometHashAggregate [s_county,sum] [total_sum,s_state,sum(UnscaledValue(ss_net_profit))] + ReusedExchange [s_state,s_county,sum] #3 + CometHashAggregate [sum,isEmpty] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(total_sum)] + CometExchange #11 + CometHashAggregate [total_sum] [sum,isEmpty] + CometHashAggregate [s_state,s_county,sum] [total_sum,sum(UnscaledValue(ss_net_profit))] + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt index c05e23f926..5269452a9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- Window (35) - +- * CometColumnarToRow (34) +* CometColumnarToRow (37) ++- CometTakeOrderedAndProject (36) + +- CometProject (35) + +- CometWindowExec (34) +- CometSort (33) +- CometExchange (32) +- CometHashAggregate (31) @@ -191,20 +191,20 @@ Arguments: hashpartitioning(lochierarchy#27, _w0#39, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39], [lochierarchy#27 ASC NULLS FIRST, _w0#39 ASC NULLS FIRST, total_sum#22 DESC NULLS LAST] -(34) CometColumnarToRow [codegen id : 1] +(34) CometWindowExec Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] +Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40], [rank(total_sum#22) windowspecdefinition(lochierarchy#27, _w0#39, total_sum#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#40], [lochierarchy#27, _w0#39], [total_sum#22 DESC NULLS LAST] -(35) Window -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] -Arguments: [rank(total_sum#22) windowspecdefinition(lochierarchy#27, _w0#39, total_sum#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#40], [lochierarchy#27, _w0#39], [total_sum#22 DESC NULLS LAST] - -(36) Project [codegen id : 2] -Output [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +(35) CometProject Input [6]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40] +Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] + +(36) CometTakeOrderedAndProject +Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#27 DESC NULLS LAST,CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST,rank_within_parent#40 ASC NULLS FIRST], output=[total_sum#22,i_category#23,i_class#24,lochierarchy#27,rank_within_parent#40]), [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40], 100, 0, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST, rank_within_parent#40 ASC NULLS FIRST], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] -(37) TakeOrderedAndProject +(37) CometColumnarToRow [codegen id : 1] Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST, rank_within_parent#40 ASC NULLS FIRST], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt index 6f5a5be275..91a7f81241 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -84,4 +84,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt index 11e3c03d80..ad8ab5e34a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt @@ -1,49 +1,47 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,rank_within_parent] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange [i_category] #7 + CometHashAggregate [total_sum] [i_category,sum,isEmpty] + CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange #8 + CometHashAggregate [total_sum] [sum,isEmpty] + CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt index c05e23f926..5269452a9e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -TakeOrderedAndProject (37) -+- * Project (36) - +- Window (35) - +- * CometColumnarToRow (34) +* CometColumnarToRow (37) ++- CometTakeOrderedAndProject (36) + +- CometProject (35) + +- CometWindowExec (34) +- CometSort (33) +- CometExchange (32) +- CometHashAggregate (31) @@ -191,20 +191,20 @@ Arguments: hashpartitioning(lochierarchy#27, _w0#39, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39], [lochierarchy#27 ASC NULLS FIRST, _w0#39 ASC NULLS FIRST, total_sum#22 DESC NULLS LAST] -(34) CometColumnarToRow [codegen id : 1] +(34) CometWindowExec Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] +Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40], [rank(total_sum#22) windowspecdefinition(lochierarchy#27, _w0#39, total_sum#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#40], [lochierarchy#27, _w0#39], [total_sum#22 DESC NULLS LAST] -(35) Window -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] -Arguments: [rank(total_sum#22) windowspecdefinition(lochierarchy#27, _w0#39, total_sum#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#40], [lochierarchy#27, _w0#39], [total_sum#22 DESC NULLS LAST] - -(36) Project [codegen id : 2] -Output [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +(35) CometProject Input [6]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40] +Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] + +(36) CometTakeOrderedAndProject +Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#27 DESC NULLS LAST,CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST,rank_within_parent#40 ASC NULLS FIRST], output=[total_sum#22,i_category#23,i_class#24,lochierarchy#27,rank_within_parent#40]), [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40], 100, 0, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST, rank_within_parent#40 ASC NULLS FIRST], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] -(37) TakeOrderedAndProject +(37) CometColumnarToRow [codegen id : 1] Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] -Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST, rank_within_parent#40 ASC NULLS FIRST], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt index 6f5a5be275..91a7f81241 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt @@ -1,7 +1,7 @@ - TakeOrderedAndProject [COMET: ] -+- Project - +- Window - +- CometColumnarToRow +CometColumnarToRow ++- CometTakeOrderedAndProject + +- CometProject + +- CometWindowExec +- CometSort +- CometExchange +- CometHashAggregate @@ -84,4 +84,4 @@ +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index 11e3c03d80..ad8ab5e34a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -1,49 +1,47 @@ -TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] - WholeStageCodegen (2) - Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - InputAdapter - Window [total_sum,lochierarchy,_w0] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,rank_within_parent] + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange [i_category] #7 + CometHashAggregate [total_sum] [i_category,sum,isEmpty] + CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange #8 + CometHashAggregate [total_sum] [sum,isEmpty] + CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt index 96a0ec392e..93c9d0d56c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt @@ -1,29 +1,28 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (24) ++- CometSort (23) + +- CometExchange (22) + +- CometProject (21) + +- CometWindowExec (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -116,57 +115,54 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +(21) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -(23) CometColumnarExchange +(22) CometExchange Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(24) CometSort +(23) CometSort Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +BroadcastExchange (29) ++- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter +(26) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(28) CometProject +(27) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(30) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt index 30105bb26b..653c70e3d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt @@ -1,33 +1,32 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt index 46191f59cd..603b05991a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt @@ -1,39 +1,34 @@ -WholeStageCodegen (3) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index 96a0ec392e..93c9d0d56c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -1,29 +1,28 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometColumnarExchange (23) - +- * Project (22) - +- Window (21) - +- * CometColumnarToRow (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (24) ++- CometSort (23) + +- CometExchange (22) + +- CometProject (21) + +- CometWindowExec (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -116,57 +115,54 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometColumnarToRow [codegen id : 1] +(20) CometWindowExec Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) Window -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] - -(22) Project [codegen id : 2] -Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +(21) CometProject Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] +Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -(23) CometColumnarExchange +(22) CometExchange Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(24) CometSort +(23) CometSort Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 3] +(24) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +BroadcastExchange (29) ++- * CometColumnarToRow (28) + +- CometProject (27) + +- CometFilter (26) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter +(26) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(28) CometProject +(27) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(28) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(30) BroadcastExchange +(29) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt index 30105bb26b..653c70e3d8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt @@ -1,33 +1,32 @@ CometColumnarToRow +- CometSort - +- CometColumnarExchange - +- Project - +- Window - +- CometColumnarToRow - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometProject + +- CometWindowExec + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index 46191f59cd..603b05991a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -1,39 +1,34 @@ -WholeStageCodegen (3) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - WholeStageCodegen (2) - Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - InputAdapter - Window [_w0,i_class] - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] + CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] From d9dc3919dee2c998c744512d45a52848df326325 Mon Sep 17 00:00:00 2001 From: comphead Date: Fri, 7 Nov 2025 15:50:27 -0800 Subject: [PATCH 4/6] chore: Fallback to Spark for windows --- .../scala/org/apache/comet/CometConf.scala | 2 +- docs/source/user-guide/latest/configs.md | 34 +- .../apache/comet/serde/QueryPlanSerde.scala | 5 +- .../q12.native_iceberg_compat/extended.txt | 2 +- .../q12/extended.txt | 2 +- .../q20.native_iceberg_compat/extended.txt | 2 +- .../q20/extended.txt | 2 +- .../q36.native_iceberg_compat/extended.txt | 2 +- .../q36/extended.txt | 2 +- .../q47.native_iceberg_compat/extended.txt | 6 +- .../q47/extended.txt | 6 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q49/extended.txt | 6 +- .../q51.native_iceberg_compat/extended.txt | 6 +- .../q51/extended.txt | 6 +- .../q53.native_iceberg_compat/extended.txt | 2 +- .../q53/extended.txt | 2 +- .../q57.native_iceberg_compat/extended.txt | 6 +- .../q57/extended.txt | 6 +- .../q63.native_iceberg_compat/extended.txt | 2 +- .../q63/extended.txt | 2 +- .../q70.native_iceberg_compat/extended.txt | 2 +- .../q70/extended.txt | 2 +- .../q86.native_iceberg_compat/extended.txt | 2 +- .../q86/extended.txt | 2 +- .../q89.native_iceberg_compat/extended.txt | 2 +- .../q89/extended.txt | 2 +- .../q98.native_iceberg_compat/extended.txt | 2 +- .../q98/extended.txt | 2 +- .../q12.native_iceberg_compat/explain.txt | 26 +- .../q12.native_iceberg_compat/extended.txt | 10 +- .../q12.native_iceberg_compat/simplified.txt | 74 ++- .../q12/explain.txt | 26 +- .../q12/extended.txt | 10 +- .../q12/simplified.txt | 74 ++- .../q20.native_iceberg_compat/explain.txt | 26 +- .../q20.native_iceberg_compat/extended.txt | 10 +- .../q20.native_iceberg_compat/simplified.txt | 74 ++- .../q20/explain.txt | 26 +- .../q20/extended.txt | 10 +- .../q20/simplified.txt | 74 ++- .../q36.native_iceberg_compat/explain.txt | 26 +- .../q36.native_iceberg_compat/extended.txt | 10 +- .../q36.native_iceberg_compat/simplified.txt | 88 +-- .../q36/explain.txt | 26 +- .../q36/extended.txt | 10 +- .../q36/simplified.txt | 88 +-- .../q47.native_iceberg_compat/explain.txt | 214 +++--- .../q47.native_iceberg_compat/extended.txt | 154 ++--- .../q47.native_iceberg_compat/simplified.txt | 130 ++-- .../q47/explain.txt | 214 +++--- .../q47/extended.txt | 154 ++--- .../q47/simplified.txt | 130 ++-- .../q49.native_iceberg_compat/explain.txt | 327 +++++----- .../q49.native_iceberg_compat/extended.txt | 177 ++--- .../q49.native_iceberg_compat/simplified.txt | 189 +++--- .../q49/explain.txt | 327 +++++----- .../q49/extended.txt | 177 ++--- .../q49/simplified.txt | 189 +++--- .../q51.native_iceberg_compat/explain.txt | 186 +++--- .../q51.native_iceberg_compat/extended.txt | 94 +-- .../q51.native_iceberg_compat/simplified.txt | 124 ++-- .../q51/explain.txt | 186 +++--- .../q51/extended.txt | 94 +-- .../q51/simplified.txt | 124 ++-- .../q53.native_iceberg_compat/explain.txt | 30 +- .../q53.native_iceberg_compat/extended.txt | 12 +- .../q53.native_iceberg_compat/simplified.txt | 86 +-- .../q53/explain.txt | 30 +- .../q53/extended.txt | 12 +- .../q53/simplified.txt | 86 +-- .../q57.native_iceberg_compat/explain.txt | 214 +++--- .../q57.native_iceberg_compat/extended.txt | 154 ++--- .../q57.native_iceberg_compat/simplified.txt | 130 ++-- .../q57/explain.txt | 214 +++--- .../q57/extended.txt | 154 ++--- .../q57/simplified.txt | 130 ++-- .../q63.native_iceberg_compat/explain.txt | 30 +- .../q63.native_iceberg_compat/extended.txt | 12 +- .../q63.native_iceberg_compat/simplified.txt | 86 +-- .../q63/explain.txt | 30 +- .../q63/extended.txt | 12 +- .../q63/simplified.txt | 86 +-- .../q70.native_iceberg_compat/explain.txt | 26 +- .../q70.native_iceberg_compat/extended.txt | 10 +- .../q70.native_iceberg_compat/simplified.txt | 146 +++-- .../q70/explain.txt | 26 +- .../q70/extended.txt | 10 +- .../q70/simplified.txt | 146 +++-- .../q86.native_iceberg_compat/explain.txt | 26 +- .../q86.native_iceberg_compat/extended.txt | 10 +- .../q86.native_iceberg_compat/simplified.txt | 76 +-- .../q86/explain.txt | 26 +- .../q86/extended.txt | 10 +- .../q86/simplified.txt | 76 +-- .../q89.native_iceberg_compat/explain.txt | 30 +- .../q89.native_iceberg_compat/extended.txt | 12 +- .../q89.native_iceberg_compat/simplified.txt | 86 +-- .../q89/explain.txt | 30 +- .../q89/extended.txt | 12 +- .../q89/simplified.txt | 86 +-- .../q98.native_iceberg_compat/explain.txt | 94 +-- .../q98.native_iceberg_compat/extended.txt | 61 +- .../q98.native_iceberg_compat/simplified.txt | 73 ++- .../q98/explain.txt | 94 +-- .../q98/extended.txt | 61 +- .../q98/simplified.txt | 73 ++- .../q12.native_iceberg_compat/explain.txt | 26 +- .../q12.native_iceberg_compat/extended.txt | 10 +- .../q12.native_iceberg_compat/simplified.txt | 68 +- .../approved-plans-v1_4/q12/explain.txt | 26 +- .../approved-plans-v1_4/q12/extended.txt | 10 +- .../approved-plans-v1_4/q12/simplified.txt | 68 +- .../q20.native_iceberg_compat/explain.txt | 26 +- .../q20.native_iceberg_compat/extended.txt | 10 +- .../q20.native_iceberg_compat/simplified.txt | 68 +- .../approved-plans-v1_4/q20/explain.txt | 26 +- .../approved-plans-v1_4/q20/extended.txt | 10 +- .../approved-plans-v1_4/q20/simplified.txt | 68 +- .../q36.native_iceberg_compat/explain.txt | 26 +- .../q36.native_iceberg_compat/extended.txt | 10 +- .../q36.native_iceberg_compat/simplified.txt | 82 +-- .../approved-plans-v1_4/q36/explain.txt | 26 +- .../approved-plans-v1_4/q36/extended.txt | 10 +- .../approved-plans-v1_4/q36/simplified.txt | 82 +-- .../q44.native_iceberg_compat/explain.txt | 195 +++--- .../q44.native_iceberg_compat/extended.txt | 115 ++-- .../q44.native_iceberg_compat/simplified.txt | 103 +-- .../approved-plans-v1_4/q44/explain.txt | 195 +++--- .../approved-plans-v1_4/q44/extended.txt | 115 ++-- .../approved-plans-v1_4/q44/simplified.txt | 103 +-- .../q47.native_iceberg_compat/explain.txt | 208 +++--- .../q47.native_iceberg_compat/extended.txt | 150 ++--- .../q47.native_iceberg_compat/simplified.txt | 122 ++-- .../approved-plans-v1_4/q47/explain.txt | 208 +++--- .../approved-plans-v1_4/q47/extended.txt | 150 ++--- .../approved-plans-v1_4/q47/simplified.txt | 122 ++-- .../q49.native_iceberg_compat/explain.txt | 327 +++++----- .../q49.native_iceberg_compat/extended.txt | 177 ++--- .../q49.native_iceberg_compat/simplified.txt | 189 +++--- .../approved-plans-v1_4/q49/explain.txt | 327 +++++----- .../approved-plans-v1_4/q49/extended.txt | 177 ++--- .../approved-plans-v1_4/q49/simplified.txt | 189 +++--- .../q51.native_iceberg_compat/explain.txt | 180 ++--- .../q51.native_iceberg_compat/extended.txt | 90 +-- .../q51.native_iceberg_compat/simplified.txt | 112 ++-- .../approved-plans-v1_4/q51/explain.txt | 180 ++--- .../approved-plans-v1_4/q51/extended.txt | 90 +-- .../approved-plans-v1_4/q51/simplified.txt | 112 ++-- .../q53.native_iceberg_compat/explain.txt | 30 +- .../q53.native_iceberg_compat/extended.txt | 12 +- .../q53.native_iceberg_compat/simplified.txt | 80 +-- .../approved-plans-v1_4/q53/explain.txt | 30 +- .../approved-plans-v1_4/q53/extended.txt | 12 +- .../approved-plans-v1_4/q53/simplified.txt | 80 +-- .../q57.native_iceberg_compat/explain.txt | 208 +++--- .../q57.native_iceberg_compat/extended.txt | 150 ++--- .../q57.native_iceberg_compat/simplified.txt | 122 ++-- .../approved-plans-v1_4/q57/explain.txt | 208 +++--- .../approved-plans-v1_4/q57/extended.txt | 150 ++--- .../approved-plans-v1_4/q57/simplified.txt | 122 ++-- .../q63.native_iceberg_compat/explain.txt | 30 +- .../q63.native_iceberg_compat/extended.txt | 12 +- .../q63.native_iceberg_compat/simplified.txt | 80 +-- .../approved-plans-v1_4/q63/explain.txt | 30 +- .../approved-plans-v1_4/q63/extended.txt | 12 +- .../approved-plans-v1_4/q63/simplified.txt | 80 +-- .../q67.native_iceberg_compat/explain.txt | 24 +- .../q67.native_iceberg_compat/extended.txt | 10 +- .../q67.native_iceberg_compat/simplified.txt | 82 +-- .../approved-plans-v1_4/q67/explain.txt | 24 +- .../approved-plans-v1_4/q67/extended.txt | 10 +- .../approved-plans-v1_4/q67/simplified.txt | 82 +-- .../q70.native_iceberg_compat/explain.txt | 258 ++++---- .../q70.native_iceberg_compat/extended.txt | 110 ++-- .../q70.native_iceberg_compat/simplified.txt | 128 ++-- .../approved-plans-v1_4/q70/explain.txt | 258 ++++---- .../approved-plans-v1_4/q70/extended.txt | 110 ++-- .../approved-plans-v1_4/q70/simplified.txt | 128 ++-- .../q86.native_iceberg_compat/explain.txt | 26 +- .../q86.native_iceberg_compat/extended.txt | 10 +- .../q86.native_iceberg_compat/simplified.txt | 70 +- .../approved-plans-v1_4/q86/explain.txt | 26 +- .../approved-plans-v1_4/q86/extended.txt | 10 +- .../approved-plans-v1_4/q86/simplified.txt | 70 +- .../q89.native_iceberg_compat/explain.txt | 30 +- .../q89.native_iceberg_compat/extended.txt | 12 +- .../q89.native_iceberg_compat/simplified.txt | 80 +-- .../approved-plans-v1_4/q89/explain.txt | 30 +- .../approved-plans-v1_4/q89/extended.txt | 12 +- .../approved-plans-v1_4/q89/simplified.txt | 80 +-- .../q98.native_iceberg_compat/explain.txt | 92 +-- .../q98.native_iceberg_compat/extended.txt | 59 +- .../q98.native_iceberg_compat/simplified.txt | 67 +- .../approved-plans-v1_4/q98/explain.txt | 92 +-- .../approved-plans-v1_4/q98/extended.txt | 59 +- .../approved-plans-v1_4/q98/simplified.txt | 67 +- .../q12.native_iceberg_compat/extended.txt | 2 +- .../q12/extended.txt | 2 +- .../q20.native_iceberg_compat/extended.txt | 2 +- .../q20/extended.txt | 2 +- .../q36a.native_iceberg_compat/extended.txt | 2 +- .../q36a/extended.txt | 2 +- .../q47.native_iceberg_compat/extended.txt | 6 +- .../q47/extended.txt | 6 +- .../q49.native_iceberg_compat/extended.txt | 6 +- .../q49/extended.txt | 6 +- .../q51a.native_iceberg_compat/extended.txt | 20 +- .../q51a/extended.txt | 20 +- .../q57.native_iceberg_compat/extended.txt | 6 +- .../q57/extended.txt | 6 +- .../q70a.native_iceberg_compat/extended.txt | 2 +- .../q70a/extended.txt | 2 +- .../q86a.native_iceberg_compat/extended.txt | 2 +- .../q86a/extended.txt | 2 +- .../q98.native_iceberg_compat/extended.txt | 2 +- .../q98/extended.txt | 2 +- .../q12.native_iceberg_compat/explain.txt | 26 +- .../q12.native_iceberg_compat/extended.txt | 10 +- .../q12.native_iceberg_compat/simplified.txt | 74 ++- .../q12/explain.txt | 26 +- .../q12/extended.txt | 10 +- .../q12/simplified.txt | 74 ++- .../q20.native_iceberg_compat/explain.txt | 26 +- .../q20.native_iceberg_compat/extended.txt | 10 +- .../q20.native_iceberg_compat/simplified.txt | 74 ++- .../q20/explain.txt | 26 +- .../q20/extended.txt | 10 +- .../q20/simplified.txt | 74 ++- .../q36a.native_iceberg_compat/explain.txt | 26 +- .../q36a.native_iceberg_compat/extended.txt | 10 +- .../q36a.native_iceberg_compat/simplified.txt | 142 ++-- .../q36a/explain.txt | 26 +- .../q36a/extended.txt | 10 +- .../q36a/simplified.txt | 142 ++-- .../q47.native_iceberg_compat/explain.txt | 214 +++--- .../q47.native_iceberg_compat/extended.txt | 154 ++--- .../q47.native_iceberg_compat/simplified.txt | 130 ++-- .../q47/explain.txt | 214 +++--- .../q47/extended.txt | 154 ++--- .../q47/simplified.txt | 130 ++-- .../q49.native_iceberg_compat/explain.txt | 327 +++++----- .../q49.native_iceberg_compat/extended.txt | 177 ++--- .../q49.native_iceberg_compat/simplified.txt | 189 +++--- .../q49/explain.txt | 327 +++++----- .../q49/extended.txt | 177 ++--- .../q49/simplified.txt | 189 +++--- .../q51a.native_iceberg_compat/explain.txt | 613 ++++++++++-------- .../q51a.native_iceberg_compat/extended.txt | 419 ++++++------ .../q51a.native_iceberg_compat/simplified.txt | 204 +++--- .../q51a/explain.txt | 613 ++++++++++-------- .../q51a/extended.txt | 419 ++++++------ .../q51a/simplified.txt | 204 +++--- .../q57.native_iceberg_compat/explain.txt | 214 +++--- .../q57.native_iceberg_compat/extended.txt | 154 ++--- .../q57.native_iceberg_compat/simplified.txt | 130 ++-- .../q57/explain.txt | 214 +++--- .../q57/extended.txt | 154 ++--- .../q57/simplified.txt | 130 ++-- .../q70a.native_iceberg_compat/explain.txt | 26 +- .../q70a.native_iceberg_compat/extended.txt | 10 +- .../q70a.native_iceberg_compat/simplified.txt | 200 +++--- .../q70a/explain.txt | 26 +- .../q70a/extended.txt | 10 +- .../q70a/simplified.txt | 200 +++--- .../q86a.native_iceberg_compat/explain.txt | 26 +- .../q86a.native_iceberg_compat/extended.txt | 10 +- .../q86a.native_iceberg_compat/simplified.txt | 130 ++-- .../q86a/explain.txt | 26 +- .../q86a/extended.txt | 10 +- .../q86a/simplified.txt | 130 ++-- .../q98.native_iceberg_compat/explain.txt | 90 +-- .../q98.native_iceberg_compat/extended.txt | 61 +- .../q98.native_iceberg_compat/simplified.txt | 73 ++- .../q98/explain.txt | 90 +-- .../q98/extended.txt | 61 +- .../q98/simplified.txt | 73 ++- .../q12.native_iceberg_compat/explain.txt | 26 +- .../q12.native_iceberg_compat/extended.txt | 10 +- .../q12.native_iceberg_compat/simplified.txt | 68 +- .../approved-plans-v2_7/q12/explain.txt | 26 +- .../approved-plans-v2_7/q12/extended.txt | 10 +- .../approved-plans-v2_7/q12/simplified.txt | 68 +- .../q20.native_iceberg_compat/explain.txt | 26 +- .../q20.native_iceberg_compat/extended.txt | 10 +- .../q20.native_iceberg_compat/simplified.txt | 68 +- .../approved-plans-v2_7/q20/explain.txt | 26 +- .../approved-plans-v2_7/q20/extended.txt | 10 +- .../approved-plans-v2_7/q20/simplified.txt | 68 +- .../q36a.native_iceberg_compat/explain.txt | 26 +- .../q36a.native_iceberg_compat/extended.txt | 10 +- .../q36a.native_iceberg_compat/simplified.txt | 108 +-- .../approved-plans-v2_7/q36a/explain.txt | 26 +- .../approved-plans-v2_7/q36a/extended.txt | 10 +- .../approved-plans-v2_7/q36a/simplified.txt | 108 +-- .../q47.native_iceberg_compat/explain.txt | 208 +++--- .../q47.native_iceberg_compat/extended.txt | 150 ++--- .../q47.native_iceberg_compat/simplified.txt | 122 ++-- .../approved-plans-v2_7/q47/explain.txt | 208 +++--- .../approved-plans-v2_7/q47/extended.txt | 150 ++--- .../approved-plans-v2_7/q47/simplified.txt | 122 ++-- .../q49.native_iceberg_compat/explain.txt | 327 +++++----- .../q49.native_iceberg_compat/extended.txt | 177 ++--- .../q49.native_iceberg_compat/simplified.txt | 189 +++--- .../approved-plans-v2_7/q49/explain.txt | 327 +++++----- .../approved-plans-v2_7/q49/extended.txt | 177 ++--- .../approved-plans-v2_7/q49/simplified.txt | 189 +++--- .../q51a.native_iceberg_compat/explain.txt | 595 +++++++++-------- .../q51a.native_iceberg_compat/extended.txt | 395 +++++------ .../q51a.native_iceberg_compat/simplified.txt | 194 +++--- .../approved-plans-v2_7/q51a/explain.txt | 595 +++++++++-------- .../approved-plans-v2_7/q51a/extended.txt | 395 +++++------ .../approved-plans-v2_7/q51a/simplified.txt | 194 +++--- .../q57.native_iceberg_compat/explain.txt | 208 +++--- .../q57.native_iceberg_compat/extended.txt | 150 ++--- .../q57.native_iceberg_compat/simplified.txt | 122 ++-- .../approved-plans-v2_7/q57/explain.txt | 208 +++--- .../approved-plans-v2_7/q57/extended.txt | 150 ++--- .../approved-plans-v2_7/q57/simplified.txt | 122 ++-- .../q67a.native_iceberg_compat/explain.txt | 24 +- .../q67a.native_iceberg_compat/extended.txt | 10 +- .../q67a.native_iceberg_compat/simplified.txt | 162 ++--- .../approved-plans-v2_7/q67a/explain.txt | 24 +- .../approved-plans-v2_7/q67a/extended.txt | 10 +- .../approved-plans-v2_7/q67a/simplified.txt | 162 ++--- .../q70a.native_iceberg_compat/explain.txt | 395 ++++++----- .../q70a.native_iceberg_compat/extended.txt | 318 ++++----- .../q70a.native_iceberg_compat/simplified.txt | 164 +++-- .../approved-plans-v2_7/q70a/explain.txt | 395 ++++++----- .../approved-plans-v2_7/q70a/extended.txt | 318 ++++----- .../approved-plans-v2_7/q70a/simplified.txt | 164 +++-- .../q86a.native_iceberg_compat/explain.txt | 26 +- .../q86a.native_iceberg_compat/extended.txt | 10 +- .../q86a.native_iceberg_compat/simplified.txt | 96 +-- .../approved-plans-v2_7/q86a/explain.txt | 26 +- .../approved-plans-v2_7/q86a/extended.txt | 10 +- .../approved-plans-v2_7/q86a/simplified.txt | 96 +-- .../q98.native_iceberg_compat/explain.txt | 88 +-- .../q98.native_iceberg_compat/extended.txt | 59 +- .../q98.native_iceberg_compat/simplified.txt | 67 +- .../approved-plans-v2_7/q98/explain.txt | 88 +-- .../approved-plans-v2_7/q98/extended.txt | 59 +- .../approved-plans-v2_7/q98/simplified.txt | 67 +- 343 files changed, 17142 insertions(+), 15035 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index d44a19a072..d48d149728 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -251,7 +251,7 @@ object CometConf extends ShimCometConf { val COMET_EXEC_EXPAND_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("expand", defaultValue = true) val COMET_EXEC_WINDOW_ENABLED: ConfigEntry[Boolean] = - createExecEnabledConfig("window", defaultValue = false) + createExecEnabledConfig("window", defaultValue = true) val COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("takeOrderedAndProject", defaultValue = true) diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 4a70234f45..6caaa53b1b 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -139,23 +139,23 @@ These settings can be used to determine which parts of the plan are accelerated | Config | Description | Default Value | |--------|-------------|---------------| -| `spark.comet.exec.aggregate.enabled` | Whether to enable aggregate by default. | true | -| `spark.comet.exec.broadcastExchange.enabled` | Whether to enable broadcastExchange by default. | true | -| `spark.comet.exec.broadcastHashJoin.enabled` | Whether to enable broadcastHashJoin by default. | true | -| `spark.comet.exec.coalesce.enabled` | Whether to enable coalesce by default. | true | -| `spark.comet.exec.collectLimit.enabled` | Whether to enable collectLimit by default. | true | -| `spark.comet.exec.expand.enabled` | Whether to enable expand by default. | true | -| `spark.comet.exec.filter.enabled` | Whether to enable filter by default. | true | -| `spark.comet.exec.globalLimit.enabled` | Whether to enable globalLimit by default. | true | -| `spark.comet.exec.hashJoin.enabled` | Whether to enable hashJoin by default. | true | -| `spark.comet.exec.localLimit.enabled` | Whether to enable localLimit by default. | true | -| `spark.comet.exec.project.enabled` | Whether to enable project by default. | true | -| `spark.comet.exec.sort.enabled` | Whether to enable sort by default. | true | -| `spark.comet.exec.sortMergeJoin.enabled` | Whether to enable sortMergeJoin by default. | true | -| `spark.comet.exec.sortMergeJoinWithJoinFilter.enabled` | Experimental support for Sort Merge Join with filter | false | -| `spark.comet.exec.takeOrderedAndProject.enabled` | Whether to enable takeOrderedAndProject by default. | true | -| `spark.comet.exec.union.enabled` | Whether to enable union by default. | true | -| `spark.comet.exec.window.enabled` | Whether to enable window by default. | false | +| `spark.comet.exec.aggregate.enabled` | Whether to enable aggregate by default. | true | +| `spark.comet.exec.broadcastExchange.enabled` | Whether to enable broadcastExchange by default. | true | +| `spark.comet.exec.broadcastHashJoin.enabled` | Whether to enable broadcastHashJoin by default. | true | +| `spark.comet.exec.coalesce.enabled` | Whether to enable coalesce by default. | true | +| `spark.comet.exec.collectLimit.enabled` | Whether to enable collectLimit by default. | true | +| `spark.comet.exec.expand.enabled` | Whether to enable expand by default. | true | +| `spark.comet.exec.filter.enabled` | Whether to enable filter by default. | true | +| `spark.comet.exec.globalLimit.enabled` | Whether to enable globalLimit by default. | true | +| `spark.comet.exec.hashJoin.enabled` | Whether to enable hashJoin by default. | true | +| `spark.comet.exec.localLimit.enabled` | Whether to enable localLimit by default. | true | +| `spark.comet.exec.project.enabled` | Whether to enable project by default. | true | +| `spark.comet.exec.sort.enabled` | Whether to enable sort by default. | true | +| `spark.comet.exec.sortMergeJoin.enabled` | Whether to enable sortMergeJoin by default. | true | +| `spark.comet.exec.sortMergeJoinWithJoinFilter.enabled` | Experimental support for Sort Merge Join with filter | false | +| `spark.comet.exec.takeOrderedAndProject.enabled` | Whether to enable takeOrderedAndProject by default. | true | +| `spark.comet.exec.union.enabled` | Whether to enable union by default. | true | +| `spark.comet.exec.window.enabled` | Whether to enable window by default. | true | ## Enabling or Disabling Individual Scalar Expressions diff --git a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala index 63e18c145a..3f8de7693c 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -1125,6 +1125,9 @@ object QueryPlanSerde extends Logging with CometExprShim { case WindowExec(windowExpression, partitionSpec, orderSpec, child) if CometConf.COMET_EXEC_WINDOW_ENABLED.get(conf) => + withInfo(op, "Window expressions are not supported") + None + /* val output = child.output val winExprs: Array[WindowExpression] = windowExpression.flatMap { expr => @@ -1165,7 +1168,7 @@ object QueryPlanSerde extends Logging with CometExprShim { Some(builder.setWindow(windowBuilder).build()) } else { None - } + } */ case aggregate: BaseAggregateExec if (aggregate.isInstanceOf[HashAggregateExec] || diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt index 2708ea9eae..9ddd3aeb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/extended.txt index 2708ea9eae..9ddd3aeb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q12/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt index b08f4f6d0e..400633ed9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/extended.txt index b08f4f6d0e..400633ed9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q20/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt index 325f54c02f..31fe2cc028 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/extended.txt index 325f54c02f..31fe2cc028 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q36/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt index 1bbdb7273f..11fbf5bd11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47.native_iceberg_compat/extended.txt @@ -7,7 +7,7 @@ : : +- Filter : : +- Window : : +- Filter - : : +- Window + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -39,7 +39,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange : +- Project - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -71,7 +71,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/extended.txt index 1bbdb7273f..11fbf5bd11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q47/extended.txt @@ -7,7 +7,7 @@ : : +- Filter : : +- Window : : +- Filter - : : +- Window + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -39,7 +39,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange : +- Project - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -71,7 +71,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt index 4988a26564..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49.native_iceberg_compat/extended.txt @@ -8,7 +8,7 @@ CometColumnarToRow : +- Filter : +- Window : +- Sort - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -40,7 +40,7 @@ CometColumnarToRow : +- Filter : +- Window : +- Sort - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -67,7 +67,7 @@ CometColumnarToRow +- Filter +- Window +- Sort - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/extended.txt index 4988a26564..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q49/extended.txt @@ -8,7 +8,7 @@ CometColumnarToRow : +- Filter : +- Window : +- Sort - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -40,7 +40,7 @@ CometColumnarToRow : +- Filter : +- Window : +- Sort - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -67,7 +67,7 @@ CometColumnarToRow +- Filter +- Window +- Sort - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt index 8d4877cb50..da1ac9c0a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange @@ -9,7 +9,7 @@ :- CometSort : +- CometColumnarExchange : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -33,7 +33,7 @@ +- CometSort +- CometColumnarExchange +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/extended.txt index 8d4877cb50..da1ac9c0a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q51/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Filter - +- Window [COMET: Partitioning and sorting specifications must be the same.] + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange @@ -9,7 +9,7 @@ :- CometSort : +- CometColumnarExchange : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -33,7 +33,7 @@ +- CometSort +- CometColumnarExchange +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt index 07ad98fa22..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [COMET: ] +- Project +- Filter - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/extended.txt index 07ad98fa22..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q53/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [COMET: ] +- Project +- Filter - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt index 101b15d3ac..7bee0e61e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57.native_iceberg_compat/extended.txt @@ -7,7 +7,7 @@ : : +- Filter : : +- Window : : +- Filter - : : +- Window + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -39,7 +39,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -71,7 +71,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center +- BroadcastExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/extended.txt index 101b15d3ac..7bee0e61e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q57/extended.txt @@ -7,7 +7,7 @@ : : +- Filter : : +- Window : : +- Filter - : : +- Window + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -39,7 +39,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -71,7 +71,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center +- BroadcastExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt index 07ad98fa22..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [COMET: ] +- Project +- Filter - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/extended.txt index 07ad98fa22..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q63/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [COMET: ] +- Project +- Filter - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt index 090d6711fc..010caafdc0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometColumnarExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt index 090d6711fc..010caafdc0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q70/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometColumnarExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt index d5ebd5a7e7..0b05b63e05 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/extended.txt index d5ebd5a7e7..0b05b63e05 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q86/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt index 07ad98fa22..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [COMET: ] +- Project +- Filter - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/extended.txt index 07ad98fa22..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q89/extended.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [COMET: ] +- Project +- Filter - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt index 8a83f62759..c34e3cfede 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98.native_iceberg_compat/extended.txt @@ -3,7 +3,7 @@ CometColumnarToRow +- CometSort +- CometColumnarExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/extended.txt index 8a83f62759..c34e3cfede 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q98/extended.txt @@ -3,7 +3,7 @@ CometColumnarToRow +- CometSort +- CometColumnarExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt index f4001679f3..866cc628a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometProject (22) - +- CometWindowExec (21) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject +(22) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(23) Project [codegen id : 3] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(23) CometTakeOrderedAndProject +(24) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -(24) CometColumnarToRow [codegen id : 2] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt index 6cb7b610ee..f99382d8c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt index 24bb6d630c..1fdffc3b73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12.native_iceberg_compat/simplified.txt @@ -1,36 +1,38 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (3) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt index f4001679f3..866cc628a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometProject (22) - +- CometWindowExec (21) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject +(22) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(23) Project [codegen id : 3] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(23) CometTakeOrderedAndProject +(24) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -(24) CometColumnarToRow [codegen id : 2] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt index 6cb7b610ee..f99382d8c3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt index 24bb6d630c..1fdffc3b73 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q12/simplified.txt @@ -1,36 +1,38 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (3) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt index 38b20b7f0c..88752d6e12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometProject (22) - +- CometWindowExec (21) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject +(22) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(23) Project [codegen id : 3] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(23) CometTakeOrderedAndProject +(24) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -(24) CometColumnarToRow [codegen id : 2] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt index f856bbd3b3..b45bca05b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt index 2cb011147e..db00a01c2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20.native_iceberg_compat/simplified.txt @@ -1,36 +1,38 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (3) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt index 38b20b7f0c..88752d6e12 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometProject (22) - +- CometWindowExec (21) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject +(22) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(23) Project [codegen id : 3] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(23) CometTakeOrderedAndProject +(24) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] - -(24) CometColumnarToRow [codegen id : 2] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt index f856bbd3b3..b45bca05b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt index 2cb011147e..db00a01c2d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q20/simplified.txt @@ -1,36 +1,38 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (3) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt index aedafeeaf5..3c6285353c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometProject (29) - +- CometWindowExec (28) +TakeOrderedAndProject (31) ++- * Project (30) + +- Window (29) + +- * CometColumnarToRow (28) +- CometSort (27) +- CometColumnarExchange (26) +- * HashAggregate (25) @@ -159,20 +159,20 @@ Arguments: hashpartitioning(_w1#26, _w2#27, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27], [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 ASC NULLS FIRST] -(28) CometWindowExec +(28) CometColumnarToRow [codegen id : 2] Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28], [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 ASC NULLS FIRST] -(29) CometProject -Input [8]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] -Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +(29) Window +Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 ASC NULLS FIRST] -(30) CometTakeOrderedAndProject -Input [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#24 DESC NULLS LAST,CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST,rank_within_parent#28 ASC NULLS FIRST], output=[gross_margin#23,i_category#16,i_class#17,lochierarchy#24,rank_within_parent#28]), [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28], 100, 0, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +(30) Project [codegen id : 3] +Output [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +Input [8]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] -(31) CometColumnarToRow [codegen id : 2] +(31) TakeOrderedAndProject Input [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +Arguments: 100, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt index 63b74d54c8..3e8236fe57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: Unsupported result expressions found in: List((MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS gross_margin#5, i_category#6, i_class#7, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS lochierarchy#9, (MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS _w0#10, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS _w1#11, CASE WHEN (cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint) = 0) THEN i_category#6 END AS _w2#12)] @@ -36,4 +36,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 31 out of 34 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 34 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt index 2b4c1272a1..6e4a6a969b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36.native_iceberg_compat/simplified.txt @@ -1,43 +1,45 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] - CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (3) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt index aedafeeaf5..3c6285353c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (31) -+- CometTakeOrderedAndProject (30) - +- CometProject (29) - +- CometWindowExec (28) +TakeOrderedAndProject (31) ++- * Project (30) + +- Window (29) + +- * CometColumnarToRow (28) +- CometSort (27) +- CometColumnarExchange (26) +- * HashAggregate (25) @@ -159,20 +159,20 @@ Arguments: hashpartitioning(_w1#26, _w2#27, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27], [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 ASC NULLS FIRST] -(28) CometWindowExec +(28) CometColumnarToRow [codegen id : 2] Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28], [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 ASC NULLS FIRST] -(29) CometProject -Input [8]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] -Arguments: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +(29) Window +Input [7]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27] +Arguments: [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 ASC NULLS FIRST] -(30) CometTakeOrderedAndProject -Input [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#24 DESC NULLS LAST,CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST,rank_within_parent#28 ASC NULLS FIRST], output=[gross_margin#23,i_category#16,i_class#17,lochierarchy#24,rank_within_parent#28]), [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28], 100, 0, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +(30) Project [codegen id : 3] +Output [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +Input [8]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] -(31) CometColumnarToRow [codegen id : 2] +(31) TakeOrderedAndProject Input [5]: [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] +Arguments: 100, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [gross_margin#23, i_category#16, i_class#17, lochierarchy#24, rank_within_parent#28] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt index 63b74d54c8..3e8236fe57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: Unsupported result expressions found in: List((MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS gross_margin#5, i_category#6, i_class#7, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS lochierarchy#9, (MakeDecimal(sum(UnscaledValue(ss_net_profit#1))#2,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#4,17,2)) AS _w0#10, (cast((shiftright(spark_grouping_id#8, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint)) AS _w1#11, CASE WHEN (cast((shiftright(spark_grouping_id#8, 0) & 1) as tinyint) = 0) THEN i_category#6 END AS _w2#12)] @@ -36,4 +36,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 31 out of 34 eligible operators (91%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 28 out of 34 eligible operators (82%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt index 2b4c1272a1..6e4a6a969b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q36/simplified.txt @@ -1,43 +1,45 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] - CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (3) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,lochierarchy,_w0,_w1,_w2,sum,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt index ab01108c6c..52a3eee555 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/explain.txt @@ -1,48 +1,54 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (29) - : : +- CometFilter (28) - : : +- CometWindowExec (27) - : : +- CometFilter (26) - : : +- CometWindowExec (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometWindowExec (35) - : +- CometSort (34) - : +- CometColumnarExchange (33) - : +- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- ReusedExchange (30) - +- ReusedExchange (40) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -160,112 +166,136 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 2] Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometFilter +(26) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(27) Filter [codegen id : 3] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) CometWindowExec +(28) Window Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22], [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(28) CometFilter +(29) Filter [codegen id : 10] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(29) CometProject +(30) Project [codegen id : 10] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -(30) ReusedExchange [Reuses operator id: 20] +(31) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(31) CometColumnarToRow [codegen id : 2] +(32) CometColumnarToRow [codegen id : 4] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(32) HashAggregate [codegen id : 2] +(33) HashAggregate [codegen id : 4] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] -(33) CometColumnarExchange +(34) CometColumnarExchange Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(34) CometSort +(35) CometSort Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(35) CometWindowExec +(36) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] + +(37) Window Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32], [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(36) CometProject +(38) Project [codegen id : 6] +Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(37) CometBroadcastExchange +(39) BroadcastExchange Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(38) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Right output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)], Inner, BuildRight +(40) BroadcastHashJoin [codegen id : 10] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] +Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)] +Join type: Inner +Join condition: None -(39) CometProject +(41) Project [codegen id : 10] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -(40) ReusedExchange [Reuses operator id: 37] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +(42) ReusedExchange [Reuses operator id: 34] +Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] + +(43) CometSort +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] + +(44) CometColumnarToRow [codegen id : 8] +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] + +(45) Window +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] + +(46) Project [codegen id : 9] +Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] -(41) CometBroadcastHashJoin -Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -Right output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#38 - 1)], Inner, BuildRight +(47) BroadcastExchange +Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(42) CometProject -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#37 AS nsum#40] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] +Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 1)] +Join type: Inner +Join condition: None -(43) CometTakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,s_store_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,s_store_name#15,s_company_name#16,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(49) Project [codegen id : 10] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -(44) CometColumnarToRow [codegen id : 4] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(50) TakeOrderedAndProject +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(52) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(47) CometColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(48) BroadcastExchange +(54) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt index eeae7c1e95..1a4201332c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/extended.txt @@ -1,78 +1,80 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] @@ -103,4 +105,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt index a72267567e..0c8d77b89c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47.native_iceberg_compat/simplified.txt @@ -1,57 +1,77 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 - WholeStageCodegen (2) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] + WholeStageCodegen (10) + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (3) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (2) CometColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt index ab01108c6c..52a3eee555 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/explain.txt @@ -1,48 +1,54 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (29) - : : +- CometFilter (28) - : : +- CometWindowExec (27) - : : +- CometFilter (26) - : : +- CometWindowExec (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometWindowExec (35) - : +- CometSort (34) - : +- CometColumnarExchange (33) - : +- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- ReusedExchange (30) - +- ReusedExchange (40) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -160,112 +166,136 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 2] Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometFilter +(26) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(27) Filter [codegen id : 3] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) CometWindowExec +(28) Window Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22], [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(28) CometFilter +(29) Filter [codegen id : 10] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(29) CometProject +(30) Project [codegen id : 10] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -(30) ReusedExchange [Reuses operator id: 20] +(31) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(31) CometColumnarToRow [codegen id : 2] +(32) CometColumnarToRow [codegen id : 4] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(32) HashAggregate [codegen id : 2] +(33) HashAggregate [codegen id : 4] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] -(33) CometColumnarExchange +(34) CometColumnarExchange Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(34) CometSort +(35) CometSort Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(35) CometWindowExec +(36) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] + +(37) Window Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32], [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(36) CometProject +(38) Project [codegen id : 6] +Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(37) CometBroadcastExchange +(39) BroadcastExchange Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(38) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Right output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)], Inner, BuildRight +(40) BroadcastHashJoin [codegen id : 10] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] +Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)] +Join type: Inner +Join condition: None -(39) CometProject +(41) Project [codegen id : 10] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -(40) ReusedExchange [Reuses operator id: 37] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +(42) ReusedExchange [Reuses operator id: 34] +Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] + +(43) CometSort +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] + +(44) CometColumnarToRow [codegen id : 8] +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] + +(45) Window +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] + +(46) Project [codegen id : 9] +Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] -(41) CometBroadcastHashJoin -Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -Right output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#38 - 1)], Inner, BuildRight +(47) BroadcastExchange +Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(42) CometProject -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#37 AS nsum#40] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] +Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 1)] +Join type: Inner +Join condition: None -(43) CometTakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,s_store_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,s_store_name#15,s_company_name#16,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(49) Project [codegen id : 10] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -(44) CometColumnarToRow [codegen id : 4] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(50) TakeOrderedAndProject +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(52) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(47) CometColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(48) BroadcastExchange +(54) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt index eeae7c1e95..1a4201332c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/extended.txt @@ -1,78 +1,80 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] @@ -103,4 +105,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt index a72267567e..0c8d77b89c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q47/simplified.txt @@ -1,57 +1,77 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 - WholeStageCodegen (2) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] + WholeStageCodegen (10) + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (3) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (2) CometColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt index 9c336d7e3c..338f5f4201 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/explain.txt @@ -1,79 +1,82 @@ == Physical Plan == -* CometColumnarToRow (75) -+- CometTakeOrderedAndProject (74) - +- CometHashAggregate (73) - +- CometExchange (72) - +- CometHashAggregate (71) - +- CometUnion (70) - :- CometProject (25) - : +- CometFilter (24) - : +- CometWindowExec (23) - : +- CometSort (22) - : +- CometWindowExec (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- CometProject (47) - : +- CometFilter (46) - : +- CometWindowExec (45) - : +- CometSort (44) - : +- CometWindowExec (43) - : +- CometSort (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometBroadcastExchange (29) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (35) - +- CometProject (69) - +- CometFilter (68) - +- CometWindowExec (67) - +- CometSort (66) - +- CometWindowExec (65) - +- CometSort (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometProject (56) - : +- CometBroadcastHashJoin (55) - : :- CometBroadcastExchange (51) - : : +- CometProject (50) - : : +- CometFilter (49) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) - +- ReusedExchange (57) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -170,27 +173,30 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) CometSort +(22) Window +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] + +(23) Sort [codegen id : 2] Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 -(23) CometWindowExec +(24) Window Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(24) CometFilter +(25) Filter [codegen id : 3] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(25) CometProject +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -198,97 +204,100 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(28) CometProject +(29) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(29) CometBroadcastExchange +(30) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(32) CometProject +(33) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(33) CometBroadcastHashJoin +(34) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(34) CometProject +(35) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(35) ReusedExchange [Reuses operator id: 13] +(36) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(36) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(37) CometProject +(38) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(38) CometHashAggregate +(39) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(39) CometExchange +(40) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) CometHashAggregate +(41) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(41) CometExchange +(42) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(42) CometSort +(43) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(43) CometWindowExec +(44) CometColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] + +(45) Window Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(44) CometSort +(46) Sort [codegen id : 5] Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 -(45) CometWindowExec +(47) Window Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(46) CometFilter +(48) Filter [codegen id : 6] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(47) CometProject +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -296,156 +305,158 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(49) CometFilter +(51) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(50) CometProject +(52) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(51) CometBroadcastExchange +(53) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(54) CometProject +(56) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(56) CometProject +(58) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(57) ReusedExchange [Reuses operator id: 13] +(59) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(59) CometProject +(61) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(60) CometHashAggregate +(62) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(61) CometExchange +(63) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(62) CometHashAggregate +(64) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(65) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(64) CometSort +(66) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(65) CometWindowExec +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(66) CometSort +(68) Window +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] + +(69) Sort [codegen id : 8] Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 -(67) CometWindowExec +(70) Window Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(68) CometFilter +(71) Filter [codegen id : 9] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(69) CometProject +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(70) CometUnion -Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(73) Union -(71) CometHashAggregate +(74) HashAggregate [codegen id : 10] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(72) CometExchange +(75) CometColumnarExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometHashAggregate +(76) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(74) CometTakeOrderedAndProject +(77) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter +(80) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(78) CometProject +(81) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(79) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(80) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt index 6a8de7fe28..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/extended.txt @@ -1,91 +1,94 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- Filter + +- Window + +- Sort + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt index 4f128e83de..ca80833ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49.native_iceberg_compat/simplified.txt @@ -1,87 +1,110 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometUnion [channel,item,return_ratio,return_rank,currency_rank] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt index 9c336d7e3c..338f5f4201 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/explain.txt @@ -1,79 +1,82 @@ == Physical Plan == -* CometColumnarToRow (75) -+- CometTakeOrderedAndProject (74) - +- CometHashAggregate (73) - +- CometExchange (72) - +- CometHashAggregate (71) - +- CometUnion (70) - :- CometProject (25) - : +- CometFilter (24) - : +- CometWindowExec (23) - : +- CometSort (22) - : +- CometWindowExec (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- CometProject (47) - : +- CometFilter (46) - : +- CometWindowExec (45) - : +- CometSort (44) - : +- CometWindowExec (43) - : +- CometSort (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometBroadcastExchange (29) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (35) - +- CometProject (69) - +- CometFilter (68) - +- CometWindowExec (67) - +- CometSort (66) - +- CometWindowExec (65) - +- CometSort (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometProject (56) - : +- CometBroadcastHashJoin (55) - : :- CometBroadcastExchange (51) - : : +- CometProject (50) - : : +- CometFilter (49) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) - +- ReusedExchange (57) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -170,27 +173,30 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) CometSort +(22) Window +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] + +(23) Sort [codegen id : 2] Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 -(23) CometWindowExec +(24) Window Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(24) CometFilter +(25) Filter [codegen id : 3] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(25) CometProject +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -198,97 +204,100 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(28) CometProject +(29) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(29) CometBroadcastExchange +(30) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(32) CometProject +(33) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(33) CometBroadcastHashJoin +(34) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(34) CometProject +(35) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(35) ReusedExchange [Reuses operator id: 13] +(36) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(36) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(37) CometProject +(38) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(38) CometHashAggregate +(39) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(39) CometExchange +(40) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) CometHashAggregate +(41) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(41) CometExchange +(42) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(42) CometSort +(43) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(43) CometWindowExec +(44) CometColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] + +(45) Window Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(44) CometSort +(46) Sort [codegen id : 5] Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 -(45) CometWindowExec +(47) Window Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(46) CometFilter +(48) Filter [codegen id : 6] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(47) CometProject +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -296,156 +305,158 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(49) CometFilter +(51) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(50) CometProject +(52) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(51) CometBroadcastExchange +(53) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(54) CometProject +(56) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(56) CometProject +(58) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(57) ReusedExchange [Reuses operator id: 13] +(59) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(59) CometProject +(61) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(60) CometHashAggregate +(62) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(61) CometExchange +(63) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(62) CometHashAggregate +(64) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(65) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(64) CometSort +(66) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(65) CometWindowExec +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(66) CometSort +(68) Window +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] + +(69) Sort [codegen id : 8] Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 -(67) CometWindowExec +(70) Window Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(68) CometFilter +(71) Filter [codegen id : 9] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(69) CometProject +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(70) CometUnion -Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(73) Union -(71) CometHashAggregate +(74) HashAggregate [codegen id : 10] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(72) CometExchange +(75) CometColumnarExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometHashAggregate +(76) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(74) CometTakeOrderedAndProject +(77) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter +(80) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(78) CometProject +(81) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(79) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(80) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt index 6a8de7fe28..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/extended.txt @@ -1,91 +1,94 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- Filter + +- Window + +- Sort + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt index 4f128e83de..ca80833ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q49/simplified.txt @@ -1,87 +1,110 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometUnion [channel,item,return_ratio,return_rank,currency_rank] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt index c6e6ed1565..bb1b8f3167 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/explain.txt @@ -1,45 +1,47 @@ == Physical Plan == -* CometColumnarToRow (41) -+- CometTakeOrderedAndProject (40) - +- CometFilter (39) - +- CometWindowExec (38) - +- CometSort (37) - +- CometExchange (36) - +- CometProject (35) - +- CometSortMergeJoin (34) - :- CometSort (18) - : +- CometExchange (17) - : +- CometProject (16) - : +- CometWindowExec (15) - : +- CometSort (14) - : +- CometColumnarExchange (13) - : +- * HashAggregate (12) - : +- * CometColumnarToRow (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (33) - +- CometExchange (32) - +- CometProject (31) - +- CometWindowExec (30) - +- CometSort (29) - +- CometColumnarExchange (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) +TakeOrderedAndProject (43) ++- * Filter (42) + +- Window (41) + +- * CometColumnarToRow (40) + +- CometSort (39) + +- CometExchange (38) + +- CometProject (37) + +- CometSortMergeJoin (36) + :- CometSort (19) + : +- CometColumnarExchange (18) + : +- * Project (17) + : +- Window (16) + : +- * CometColumnarToRow (15) + : +- CometSort (14) + : +- CometColumnarExchange (13) + : +- * HashAggregate (12) + : +- * CometColumnarToRow (11) + : +- CometExchange (10) + : +- CometHashAggregate (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- CometSort (35) + +- CometColumnarExchange (34) + +- * Project (33) + +- Window (32) + +- * CometColumnarToRow (31) + +- CometSort (30) + +- CometColumnarExchange (29) + +- * HashAggregate (28) + +- * CometColumnarToRow (27) + +- CometExchange (26) + +- CometHashAggregate (25) + +- CometProject (24) + +- CometBroadcastHashJoin (23) + :- CometFilter (21) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (20) + +- ReusedExchange (22) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -109,23 +111,26 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) CometWindowExec +(15) CometColumnarToRow [codegen id : 2] Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12], [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(16) CometProject +(16) Window +Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] + +(17) Project [codegen id : 3] +Output [3]: [item_sk#10, d_date#6, cume_sales#12] Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] -Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10, d_date#6, cume_sales#12] -(17) CometExchange +(18) CometColumnarExchange Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) CometSort +(19) CometSort Input [3]: [item_sk#10, d_date#6, cume_sales#12] Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] @@ -133,129 +138,132 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(20) CometFilter +(21) CometFilter Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Condition : isnotnull(ss_item_sk#13) -(21) ReusedExchange [Reuses operator id: 6] +(22) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#17, d_date#18] -(22) CometBroadcastHashJoin +(23) CometBroadcastHashJoin Left output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Right output [2]: [d_date_sk#17, d_date#18] Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight -(23) CometProject +(24) CometProject Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] Arguments: [ss_item_sk#13, ss_sales_price#14, d_date#18], [ss_item_sk#13, ss_sales_price#14, d_date#18] -(24) CometHashAggregate +(25) CometHashAggregate Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -(25) CometExchange +(26) CometExchange Input [3]: [ss_item_sk#13, d_date#18, sum#19] Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(26) CometColumnarToRow [codegen id : 2] +(27) CometColumnarToRow [codegen id : 4] Input [3]: [ss_item_sk#13, d_date#18, sum#19] -(27) HashAggregate [codegen id : 2] +(28) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#13, d_date#18, sum#19] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] Results [4]: [ss_item_sk#13 AS item_sk#21, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS _w0#22, ss_item_sk#13] -(28) CometColumnarExchange +(29) CometColumnarExchange Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(29) CometSort +(30) CometSort Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] -(30) CometWindowExec +(31) CometColumnarToRow [codegen id : 5] +Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] + +(32) Window Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] -Arguments: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23], [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] +Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] -(31) CometProject +(33) Project [codegen id : 6] +Output [3]: [item_sk#21, d_date#18, cume_sales#23] Input [5]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23] -Arguments: [item_sk#21, d_date#18, cume_sales#23], [item_sk#21, d_date#18, cume_sales#23] -(32) CometExchange +(34) CometColumnarExchange Input [3]: [item_sk#21, d_date#18, cume_sales#23] -Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(33) CometSort +(35) CometSort Input [3]: [item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#21, d_date#18, cume_sales#23], [item_sk#21 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] -(34) CometSortMergeJoin +(36) CometSortMergeJoin Left output [3]: [item_sk#10, d_date#6, cume_sales#12] Right output [3]: [item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#10, d_date#6], [item_sk#21, d_date#18], FullOuter -(35) CometProject +(37) CometProject Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#21 END AS item_sk#24, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#25, cume_sales#12 AS web_sales#26, cume_sales#23 AS store_sales#27] -(36) CometExchange +(38) CometExchange Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(37) CometSort +(39) CometSort Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] -(38) CometWindowExec +(40) CometColumnarToRow [codegen id : 7] Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29], [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] -(39) CometFilter -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) +(41) Window +Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] -(40) CometTakeOrderedAndProject +(42) Filter [codegen id : 8] Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#24 ASC NULLS FIRST,d_date#25 ASC NULLS FIRST], output=[item_sk#24,d_date#25,web_sales#26,store_sales#27,web_cumulative#28,store_cumulative#29]), [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29], 100, 0, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) -(41) CometColumnarToRow [codegen id : 3] +(43) TakeOrderedAndProject Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter +(45) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(44) CometProject +(46) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(45) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(46) BroadcastExchange +(48) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt index ff86e0f44c..d5de75daf6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/extended.txt @@ -1,53 +1,55 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS _w0#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS _w0#6, ws_item_sk#1)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS _w0#12, ss_item_sk#7)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS _w0#12, ss_item_sk#7)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 42 out of 47 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 47 eligible operators (74%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt index 493a61c885..dc6a7ea61f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51.native_iceberg_compat/simplified.txt @@ -1,56 +1,68 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometWindowExec [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #2 - CometProject [item_sk,d_date,cume_sales] - CometWindowExec [item_sk,d_date,_w0,ws_item_sk,cume_sales] - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #7 - CometProject [item_sk,d_date,cume_sales] - CometWindowExec [item_sk,d_date,_w0,ss_item_sk,cume_sales] - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #8 - WholeStageCodegen (2) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #6 +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (8) + Filter [web_cumulative,store_cumulative] + InputAdapter + Window [web_sales,item_sk,d_date,store_sales] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (3) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [ws_item_sk,d_date] #4 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #6 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #7 + WholeStageCodegen (6) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometColumnarExchange [ss_item_sk] #8 + WholeStageCodegen (4) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [ss_item_sk,d_date] #9 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt index c6e6ed1565..bb1b8f3167 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/explain.txt @@ -1,45 +1,47 @@ == Physical Plan == -* CometColumnarToRow (41) -+- CometTakeOrderedAndProject (40) - +- CometFilter (39) - +- CometWindowExec (38) - +- CometSort (37) - +- CometExchange (36) - +- CometProject (35) - +- CometSortMergeJoin (34) - :- CometSort (18) - : +- CometExchange (17) - : +- CometProject (16) - : +- CometWindowExec (15) - : +- CometSort (14) - : +- CometColumnarExchange (13) - : +- * HashAggregate (12) - : +- * CometColumnarToRow (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (33) - +- CometExchange (32) - +- CometProject (31) - +- CometWindowExec (30) - +- CometSort (29) - +- CometColumnarExchange (28) - +- * HashAggregate (27) - +- * CometColumnarToRow (26) - +- CometExchange (25) - +- CometHashAggregate (24) - +- CometProject (23) - +- CometBroadcastHashJoin (22) - :- CometFilter (20) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) - +- ReusedExchange (21) +TakeOrderedAndProject (43) ++- * Filter (42) + +- Window (41) + +- * CometColumnarToRow (40) + +- CometSort (39) + +- CometExchange (38) + +- CometProject (37) + +- CometSortMergeJoin (36) + :- CometSort (19) + : +- CometColumnarExchange (18) + : +- * Project (17) + : +- Window (16) + : +- * CometColumnarToRow (15) + : +- CometSort (14) + : +- CometColumnarExchange (13) + : +- * HashAggregate (12) + : +- * CometColumnarToRow (11) + : +- CometExchange (10) + : +- CometHashAggregate (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- CometSort (35) + +- CometColumnarExchange (34) + +- * Project (33) + +- Window (32) + +- * CometColumnarToRow (31) + +- CometSort (30) + +- CometColumnarExchange (29) + +- * HashAggregate (28) + +- * CometColumnarToRow (27) + +- CometExchange (26) + +- CometHashAggregate (25) + +- CometProject (24) + +- CometBroadcastHashJoin (23) + :- CometFilter (21) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (20) + +- ReusedExchange (22) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -109,23 +111,26 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) CometWindowExec +(15) CometColumnarToRow [codegen id : 2] Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12], [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(16) CometProject +(16) Window +Input [4]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1] +Arguments: [sum(_w0#11) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] + +(17) Project [codegen id : 3] +Output [3]: [item_sk#10, d_date#6, cume_sales#12] Input [5]: [item_sk#10, d_date#6, _w0#11, ws_item_sk#1, cume_sales#12] -Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10, d_date#6, cume_sales#12] -(17) CometExchange +(18) CometColumnarExchange Input [3]: [item_sk#10, d_date#6, cume_sales#12] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(18) CometSort +(19) CometSort Input [3]: [item_sk#10, d_date#6, cume_sales#12] Arguments: [item_sk#10, d_date#6, cume_sales#12], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(20) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Batched: true Location: InMemoryFileIndex [] @@ -133,129 +138,132 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#15), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(20) CometFilter +(21) CometFilter Input [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Condition : isnotnull(ss_item_sk#13) -(21) ReusedExchange [Reuses operator id: 6] +(22) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#17, d_date#18] -(22) CometBroadcastHashJoin +(23) CometBroadcastHashJoin Left output [3]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15] Right output [2]: [d_date_sk#17, d_date#18] Arguments: [ss_sold_date_sk#15], [d_date_sk#17], Inner, BuildRight -(23) CometProject +(24) CometProject Input [5]: [ss_item_sk#13, ss_sales_price#14, ss_sold_date_sk#15, d_date_sk#17, d_date#18] Arguments: [ss_item_sk#13, ss_sales_price#14, d_date#18], [ss_item_sk#13, ss_sales_price#14, d_date#18] -(24) CometHashAggregate +(25) CometHashAggregate Input [3]: [ss_item_sk#13, ss_sales_price#14, d_date#18] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#14))] -(25) CometExchange +(26) CometExchange Input [3]: [ss_item_sk#13, d_date#18, sum#19] Arguments: hashpartitioning(ss_item_sk#13, d_date#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(26) CometColumnarToRow [codegen id : 2] +(27) CometColumnarToRow [codegen id : 4] Input [3]: [ss_item_sk#13, d_date#18, sum#19] -(27) HashAggregate [codegen id : 2] +(28) HashAggregate [codegen id : 4] Input [3]: [ss_item_sk#13, d_date#18, sum#19] Keys [2]: [ss_item_sk#13, d_date#18] Functions [1]: [sum(UnscaledValue(ss_sales_price#14))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#14))#20] Results [4]: [ss_item_sk#13 AS item_sk#21, d_date#18, MakeDecimal(sum(UnscaledValue(ss_sales_price#14))#20,17,2) AS _w0#22, ss_item_sk#13] -(28) CometColumnarExchange +(29) CometColumnarExchange Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: hashpartitioning(ss_item_sk#13, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(29) CometSort +(30) CometSort Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] Arguments: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13], [ss_item_sk#13 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] -(30) CometWindowExec +(31) CometColumnarToRow [codegen id : 5] +Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] + +(32) Window Input [4]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13] -Arguments: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23], [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] +Arguments: [sum(_w0#22) windowspecdefinition(ss_item_sk#13, d_date#18 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#23], [ss_item_sk#13], [d_date#18 ASC NULLS FIRST] -(31) CometProject +(33) Project [codegen id : 6] +Output [3]: [item_sk#21, d_date#18, cume_sales#23] Input [5]: [item_sk#21, d_date#18, _w0#22, ss_item_sk#13, cume_sales#23] -Arguments: [item_sk#21, d_date#18, cume_sales#23], [item_sk#21, d_date#18, cume_sales#23] -(32) CometExchange +(34) CometColumnarExchange Input [3]: [item_sk#21, d_date#18, cume_sales#23] -Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +Arguments: hashpartitioning(item_sk#21, d_date#18, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(33) CometSort +(35) CometSort Input [3]: [item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#21, d_date#18, cume_sales#23], [item_sk#21 ASC NULLS FIRST, d_date#18 ASC NULLS FIRST] -(34) CometSortMergeJoin +(36) CometSortMergeJoin Left output [3]: [item_sk#10, d_date#6, cume_sales#12] Right output [3]: [item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#10, d_date#6], [item_sk#21, d_date#18], FullOuter -(35) CometProject +(37) CometProject Input [6]: [item_sk#10, d_date#6, cume_sales#12, item_sk#21, d_date#18, cume_sales#23] Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#21 END AS item_sk#24, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#18 END AS d_date#25, cume_sales#12 AS web_sales#26, cume_sales#23 AS store_sales#27] -(36) CometExchange +(38) CometExchange Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: hashpartitioning(item_sk#24, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(37) CometSort +(39) CometSort Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27], [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] -(38) CometWindowExec +(40) CometColumnarToRow [codegen id : 7] Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] -Arguments: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29], [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] -(39) CometFilter -Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) +(41) Window +Input [4]: [item_sk#24, d_date#25, web_sales#26, store_sales#27] +Arguments: [max(web_sales#26) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#28, max(store_sales#27) windowspecdefinition(item_sk#24, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#29], [item_sk#24], [d_date#25 ASC NULLS FIRST] -(40) CometTakeOrderedAndProject +(42) Filter [codegen id : 8] Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#24 ASC NULLS FIRST,d_date#25 ASC NULLS FIRST], output=[item_sk#24,d_date#25,web_sales#26,store_sales#27,web_cumulative#28,store_cumulative#29]), [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29], 100, 0, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +Condition : ((isnotnull(web_cumulative#28) AND isnotnull(store_cumulative#29)) AND (web_cumulative#28 > store_cumulative#29)) -(41) CometColumnarToRow [codegen id : 3] +(43) TakeOrderedAndProject Input [6]: [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] +Arguments: 100, [item_sk#24 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST], [item_sk#24, d_date#25, web_sales#26, store_sales#27, web_cumulative#28, store_cumulative#29] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometProject (44) - +- CometFilter (43) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) +BroadcastExchange (48) ++- * CometColumnarToRow (47) + +- CometProject (46) + +- CometFilter (45) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (44) -(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(43) CometFilter +(45) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(44) CometProject +(46) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(45) CometColumnarToRow [codegen id : 1] +(47) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(46) BroadcastExchange +(48) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 20 Hosting Expression = ss_sold_date_sk#15 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt index ff86e0f44c..d5de75daf6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/extended.txt @@ -1,53 +1,55 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS _w0#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS _w0#6, ws_item_sk#1)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS _w0#12, ss_item_sk#7)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS _w0#12, ss_item_sk#7)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 42 out of 47 eligible operators (89%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 35 out of 47 eligible operators (74%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt index 493a61c885..dc6a7ea61f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q51/simplified.txt @@ -1,56 +1,68 @@ -WholeStageCodegen (3) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometWindowExec [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #2 - CometProject [item_sk,d_date,cume_sales] - CometWindowExec [item_sk,d_date,_w0,ws_item_sk,cume_sales] - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometColumnarExchange [ws_item_sk] #3 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #7 - CometProject [item_sk,d_date,cume_sales] - CometWindowExec [item_sk,d_date,_w0,ss_item_sk,cume_sales] - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometColumnarExchange [ss_item_sk] #8 - WholeStageCodegen (2) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #6 +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (8) + Filter [web_cumulative,store_cumulative] + InputAdapter + Window [web_sales,item_sk,d_date,store_sales] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (3) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometColumnarExchange [ws_item_sk] #3 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [ws_item_sk,d_date] #4 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #6 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #7 + WholeStageCodegen (6) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometColumnarExchange [ss_item_sk] #8 + WholeStageCodegen (4) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [ss_item_sk,d_date] #9 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt index fbcbd86dc1..474ac8078d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (30) -+- CometTakeOrderedAndProject (29) - +- CometProject (28) - +- CometFilter (27) - +- CometWindowExec (26) +TakeOrderedAndProject (30) ++- * Project (29) + +- * Filter (28) + +- Window (27) + +- * CometColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColum Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] -(26) CometWindowExec +(26) CometColumnarToRow [codegen id : 2] Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23], [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] -(27) CometFilter +(27) Window +Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] + +(28) Filter [codegen id : 3] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END -(28) CometProject +(29) Project [codegen id : 3] +Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Arguments: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] - -(29) CometTakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[avg_quarterly_sales#23 ASC NULLS FIRST,sum_sales#21 ASC NULLS FIRST,i_manufact_id#5 ASC NULLS FIRST], output=[i_manufact_id#5,sum_sales#21,avg_quarterly_sales#23]), [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23], 100, 0, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -(30) CometColumnarToRow [codegen id : 2] +(30) TakeOrderedAndProject Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt index 496649c274..577a05e76d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt index 428cc26f34..eab0efef6c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53.native_iceberg_compat/simplified.txt @@ -1,42 +1,44 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manufact_id,sum_sales,avg_quarterly_sales] - CometProject [i_manufact_id,sum_sales,avg_quarterly_sales] - CometFilter [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] - CometWindowExec [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] - CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] + WholeStageCodegen (3) + Project [i_manufact_id,sum_sales,avg_quarterly_sales] + Filter [avg_quarterly_sales,sum_sales] + InputAdapter + Window [_w0,i_manufact_id] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_manufact_id,sum_sales,_w0] + CometColumnarExchange [i_manufact_id] #1 + WholeStageCodegen (1) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt index fbcbd86dc1..474ac8078d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (30) -+- CometTakeOrderedAndProject (29) - +- CometProject (28) - +- CometFilter (27) - +- CometWindowExec (26) +TakeOrderedAndProject (30) ++- * Project (29) + +- * Filter (28) + +- Window (27) + +- * CometColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometColum Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] Arguments: [i_manufact_id#5, sum_sales#21, _w0#22], [i_manufact_id#5 ASC NULLS FIRST] -(26) CometWindowExec +(26) CometColumnarToRow [codegen id : 2] Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] -Arguments: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23], [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] -(27) CometFilter +(27) Window +Input [3]: [i_manufact_id#5, sum_sales#21, _w0#22] +Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#23], [i_manufact_id#5] + +(28) Filter [codegen id : 3] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] Condition : CASE WHEN (avg_quarterly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000) ELSE false END -(28) CometProject +(29) Project [codegen id : 3] +Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23] -Arguments: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] - -(29) CometTakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[avg_quarterly_sales#23 ASC NULLS FIRST,sum_sales#21 ASC NULLS FIRST,i_manufact_id#5 ASC NULLS FIRST], output=[i_manufact_id#5,sum_sales#21,avg_quarterly_sales#23]), [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23], 100, 0, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] -(30) CometColumnarToRow [codegen id : 2] +(30) TakeOrderedAndProject Input [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] +Arguments: 100, [avg_quarterly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt index 496649c274..577a05e76d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manufact_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt index 428cc26f34..eab0efef6c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q53/simplified.txt @@ -1,42 +1,44 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manufact_id,sum_sales,avg_quarterly_sales] - CometProject [i_manufact_id,sum_sales,avg_quarterly_sales] - CometFilter [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] - CometWindowExec [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] - CometSort [i_manufact_id,sum_sales,_w0] - CometColumnarExchange [i_manufact_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] + WholeStageCodegen (3) + Project [i_manufact_id,sum_sales,avg_quarterly_sales] + Filter [avg_quarterly_sales,sum_sales] + InputAdapter + Window [_w0,i_manufact_id] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_manufact_id,sum_sales,_w0] + CometColumnarExchange [i_manufact_id] #1 + WholeStageCodegen (1) + HashAggregate [i_manufact_id,d_qoy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt index ae9b283481..eb0ce14345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/explain.txt @@ -1,48 +1,54 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (29) - : : +- CometFilter (28) - : : +- CometWindowExec (27) - : : +- CometFilter (26) - : : +- CometWindowExec (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometWindowExec (35) - : +- CometSort (34) - : +- CometColumnarExchange (33) - : +- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- ReusedExchange (30) - +- ReusedExchange (40) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -160,112 +166,136 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 2] Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometFilter +(26) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(27) Filter [codegen id : 3] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) CometWindowExec +(28) Window Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(28) CometFilter +(29) Filter [codegen id : 10] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(29) CometProject +(30) Project [codegen id : 10] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(30) ReusedExchange [Reuses operator id: 20] +(31) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(31) CometColumnarToRow [codegen id : 2] +(32) CometColumnarToRow [codegen id : 4] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(32) HashAggregate [codegen id : 2] +(33) HashAggregate [codegen id : 4] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] -(33) CometColumnarExchange +(34) CometColumnarExchange Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(34) CometSort +(35) CometSort Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(35) CometWindowExec +(36) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] + +(37) Window Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30], [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(36) CometProject +(38) Project [codegen id : 6] +Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] -Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30], [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(37) CometBroadcastExchange +(39) BroadcastExchange Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(38) CometBroadcastHashJoin -Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Right output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)], Inner, BuildRight +(40) BroadcastHashJoin [codegen id : 10] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] +Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)] +Join type: Inner +Join condition: None -(39) CometProject +(41) Project [codegen id : 10] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -(40) ReusedExchange [Reuses operator id: 37] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +(42) ReusedExchange [Reuses operator id: 34] +Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] + +(43) CometSort +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(44) CometColumnarToRow [codegen id : 8] +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] + +(45) Window +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(46) Project [codegen id : 9] +Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] -(41) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -Right output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#31, i_brand#32, cc_name#33, (rn#35 - 1)], Inner, BuildRight +(47) BroadcastExchange +Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(42) CometProject -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#36, sum_sales#34 AS nsum#37] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] +Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 1)] +Join type: Inner +Join condition: None -(43) CometTakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,cc_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,cc_name#15,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#36,nsum#37]), [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +(49) Project [codegen id : 10] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -(44) CometColumnarToRow [codegen id : 4] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +(50) TakeOrderedAndProject +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(52) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(47) CometColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(48) BroadcastExchange +(54) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt index 8d50c97859..1c4e1f9890 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/extended.txt @@ -1,78 +1,80 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] @@ -103,4 +105,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt index 6ee00f9e29..dc90429f65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57.native_iceberg_compat/simplified.txt @@ -1,57 +1,77 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,cc_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (2) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] + WholeStageCodegen (10) + Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (3) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (2) CometColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt index ae9b283481..eb0ce14345 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/explain.txt @@ -1,48 +1,54 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (29) - : : +- CometFilter (28) - : : +- CometWindowExec (27) - : : +- CometFilter (26) - : : +- CometWindowExec (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometWindowExec (35) - : +- CometSort (34) - : +- CometColumnarExchange (33) - : +- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- ReusedExchange (30) - +- ReusedExchange (40) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -160,112 +166,136 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 2] Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometFilter +(26) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(27) Filter [codegen id : 3] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) CometWindowExec +(28) Window Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(28) CometFilter +(29) Filter [codegen id : 10] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(29) CometProject +(30) Project [codegen id : 10] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(30) ReusedExchange [Reuses operator id: 20] +(31) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(31) CometColumnarToRow [codegen id : 2] +(32) CometColumnarToRow [codegen id : 4] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(32) HashAggregate [codegen id : 2] +(33) HashAggregate [codegen id : 4] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] -(33) CometColumnarExchange +(34) CometColumnarExchange Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(34) CometSort +(35) CometSort Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(35) CometWindowExec +(36) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] + +(37) Window Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30], [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(36) CometProject +(38) Project [codegen id : 6] +Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] -Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30], [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(37) CometBroadcastExchange +(39) BroadcastExchange Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(38) CometBroadcastHashJoin -Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Right output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)], Inner, BuildRight +(40) BroadcastHashJoin [codegen id : 10] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] +Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)] +Join type: Inner +Join condition: None -(39) CometProject +(41) Project [codegen id : 10] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -(40) ReusedExchange [Reuses operator id: 37] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +(42) ReusedExchange [Reuses operator id: 34] +Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] + +(43) CometSort +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(44) CometColumnarToRow [codegen id : 8] +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] + +(45) Window +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(46) Project [codegen id : 9] +Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] -(41) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -Right output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#31, i_brand#32, cc_name#33, (rn#35 - 1)], Inner, BuildRight +(47) BroadcastExchange +Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(42) CometProject -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#36, sum_sales#34 AS nsum#37] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] +Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 1)] +Join type: Inner +Join condition: None -(43) CometTakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,cc_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,cc_name#15,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#36,nsum#37]), [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +(49) Project [codegen id : 10] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -(44) CometColumnarToRow [codegen id : 4] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +(50) TakeOrderedAndProject +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(52) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(47) CometColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(48) BroadcastExchange +(54) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt index 8d50c97859..1c4e1f9890 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/extended.txt @@ -1,78 +1,80 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] @@ -103,4 +105,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt index 6ee00f9e29..dc90429f65 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q57/simplified.txt @@ -1,57 +1,77 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,cc_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (2) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] + WholeStageCodegen (10) + Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (3) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (2) CometColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt index 049b5567ef..d927679b54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (30) -+- CometTakeOrderedAndProject (29) - +- CometProject (28) - +- CometFilter (27) - +- CometWindowExec (26) +TakeOrderedAndProject (30) ++- * Project (29) + +- * Filter (28) + +- Window (27) + +- * CometColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumn Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] -(26) CometWindowExec +(26) CometColumnarToRow [codegen id : 2] Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23], [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] -(27) CometFilter +(27) Window +Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] + +(28) Filter [codegen id : 3] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END -(28) CometProject +(29) Project [codegen id : 3] +Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Arguments: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] - -(29) CometTakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_manager_id#5 ASC NULLS FIRST,avg_monthly_sales#23 ASC NULLS FIRST,sum_sales#21 ASC NULLS FIRST], output=[i_manager_id#5,sum_sales#21,avg_monthly_sales#23]), [i_manager_id#5, sum_sales#21, avg_monthly_sales#23], 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -(30) CometColumnarToRow [codegen id : 2] +(30) TakeOrderedAndProject Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt index a622e150da..2d0e88e77e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manager_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt index ca60ad63cc..6688a7a467 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63.native_iceberg_compat/simplified.txt @@ -1,42 +1,44 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manager_id,sum_sales,avg_monthly_sales] - CometProject [i_manager_id,sum_sales,avg_monthly_sales] - CometFilter [i_manager_id,sum_sales,_w0,avg_monthly_sales] - CometWindowExec [i_manager_id,sum_sales,_w0,avg_monthly_sales] - CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] + WholeStageCodegen (3) + Project [i_manager_id,sum_sales,avg_monthly_sales] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_manager_id] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_manager_id,sum_sales,_w0] + CometColumnarExchange [i_manager_id] #1 + WholeStageCodegen (1) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_manager_id,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt index 049b5567ef..d927679b54 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (30) -+- CometTakeOrderedAndProject (29) - +- CometProject (28) - +- CometFilter (27) - +- CometWindowExec (26) +TakeOrderedAndProject (30) ++- * Project (29) + +- * Filter (28) + +- Window (27) + +- * CometColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometColumn Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] Arguments: [i_manager_id#5, sum_sales#21, _w0#22], [i_manager_id#5 ASC NULLS FIRST] -(26) CometWindowExec +(26) CometColumnarToRow [codegen id : 2] Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] -Arguments: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23], [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] -(27) CometFilter +(27) Window +Input [3]: [i_manager_id#5, sum_sales#21, _w0#22] +Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_manager_id#5] + +(28) Filter [codegen id : 3] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN (avg_monthly_sales#23 > 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) ELSE false END -(28) CometProject +(29) Project [codegen id : 3] +Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23] -Arguments: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] - -(29) CometTakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_manager_id#5 ASC NULLS FIRST,avg_monthly_sales#23 ASC NULLS FIRST,sum_sales#21 ASC NULLS FIRST], output=[i_manager_id#5,sum_sales#21,avg_monthly_sales#23]), [i_manager_id#5, sum_sales#21, avg_monthly_sales#23], 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] -(30) CometColumnarToRow [codegen id : 2] +(30) TakeOrderedAndProject Input [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#23 ASC NULLS FIRST, sum_sales#21 ASC NULLS FIRST], [i_manager_id#5, sum_sales#21, avg_monthly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt index a622e150da..2d0e88e77e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_manager_id#1, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS sum_sales#4, MakeDecimal(sum(UnscaledValue(ss_sales_price#2))#3,17,2) AS _w0#5)] @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt index ca60ad63cc..6688a7a467 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q63/simplified.txt @@ -1,42 +1,44 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manager_id,sum_sales,avg_monthly_sales] - CometProject [i_manager_id,sum_sales,avg_monthly_sales] - CometFilter [i_manager_id,sum_sales,_w0,avg_monthly_sales] - CometWindowExec [i_manager_id,sum_sales,_w0,avg_monthly_sales] - CometSort [i_manager_id,sum_sales,_w0] - CometColumnarExchange [i_manager_id] #1 - WholeStageCodegen (1) - HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] + WholeStageCodegen (3) + Project [i_manager_id,sum_sales,avg_monthly_sales] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_manager_id] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_manager_id,sum_sales,_w0] + CometColumnarExchange [i_manager_id] #1 + WholeStageCodegen (1) + HashAggregate [i_manager_id,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_manager_id,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt index b2d0cf6078..23cfae9635 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (49) -+- CometTakeOrderedAndProject (48) - +- CometProject (47) - +- CometWindowExec (46) +TakeOrderedAndProject (49) ++- * Project (48) + +- Window (47) + +- * CometColumnarToRow (46) +- CometSort (45) +- CometColumnarExchange (44) +- * HashAggregate (43) @@ -259,20 +259,20 @@ Arguments: hashpartitioning(_w1#32, _w2#33, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33], [_w1#32 ASC NULLS FIRST, _w2#33 ASC NULLS FIRST, _w0#31 DESC NULLS LAST] -(46) CometWindowExec +(46) CometColumnarToRow [codegen id : 6] Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] -Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34], [rank(_w0#31) windowspecdefinition(_w1#32, _w2#33, _w0#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [_w1#32, _w2#33], [_w0#31 DESC NULLS LAST] -(47) CometProject -Input [8]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34] -Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +(47) Window +Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] +Arguments: [rank(_w0#31) windowspecdefinition(_w1#32, _w2#33, _w0#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [_w1#32, _w2#33], [_w0#31 DESC NULLS LAST] -(48) CometTakeOrderedAndProject -Input [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#30 DESC NULLS LAST,CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST,rank_within_parent#34 ASC NULLS FIRST], output=[total_sum#29,s_state#23,s_county#24,lochierarchy#30,rank_within_parent#34]), [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34], 100, 0, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +(48) Project [codegen id : 7] +Output [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +Input [8]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34] -(49) CometColumnarToRow [codegen id : 6] +(49) TakeOrderedAndProject Input [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt index 3245d362b3..4cf87804d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate @@ -58,4 +58,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 53 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt index 46e164b143..f6953d47ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70.native_iceberg_compat/simplified.txt @@ -1,72 +1,74 @@ -WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #4 +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (7) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + CometColumnarToRow + InputAdapter + CometExchange [s_state] #7 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt index b2d0cf6078..23cfae9635 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (49) -+- CometTakeOrderedAndProject (48) - +- CometProject (47) - +- CometWindowExec (46) +TakeOrderedAndProject (49) ++- * Project (48) + +- Window (47) + +- * CometColumnarToRow (46) +- CometSort (45) +- CometColumnarExchange (44) +- * HashAggregate (43) @@ -259,20 +259,20 @@ Arguments: hashpartitioning(_w1#32, _w2#33, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33], [_w1#32 ASC NULLS FIRST, _w2#33 ASC NULLS FIRST, _w0#31 DESC NULLS LAST] -(46) CometWindowExec +(46) CometColumnarToRow [codegen id : 6] Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] -Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34], [rank(_w0#31) windowspecdefinition(_w1#32, _w2#33, _w0#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [_w1#32, _w2#33], [_w0#31 DESC NULLS LAST] -(47) CometProject -Input [8]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34] -Arguments: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +(47) Window +Input [7]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33] +Arguments: [rank(_w0#31) windowspecdefinition(_w1#32, _w2#33, _w0#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#34], [_w1#32, _w2#33], [_w0#31 DESC NULLS LAST] -(48) CometTakeOrderedAndProject -Input [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#30 DESC NULLS LAST,CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST,rank_within_parent#34 ASC NULLS FIRST], output=[total_sum#29,s_state#23,s_county#24,lochierarchy#30,rank_within_parent#34]), [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34], 100, 0, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +(48) Project [codegen id : 7] +Output [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +Input [8]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, _w0#31, _w1#32, _w2#33, rank_within_parent#34] -(49) CometColumnarToRow [codegen id : 6] +(49) TakeOrderedAndProject Input [5]: [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] +Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#23 END ASC NULLS FIRST, rank_within_parent#34 ASC NULLS FIRST], [total_sum#29, s_state#23, s_county#24, lochierarchy#30, rank_within_parent#34] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt index 3245d362b3..4cf87804d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate @@ -58,4 +58,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 35 out of 53 eligible operators (66%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 32 out of 53 eligible operators (60%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt index 46e164b143..f6953d47ff 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q70/simplified.txt @@ -1,72 +1,74 @@ -WholeStageCodegen (6) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (5) - HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] - Expand [ss_net_profit,s_state,s_county] - Project [ss_net_profit,s_state,s_county] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #5 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #4 +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (7) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + CometColumnarToRow + InputAdapter + CometExchange [s_state] #7 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt index f07df4bf44..de788a42e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- CometProject (23) - +- CometWindowExec (22) +TakeOrderedAndProject (25) ++- * Project (24) + +- Window (23) + +- * CometColumnarToRow (22) +- CometSort (21) +- CometColumnarExchange (20) +- * HashAggregate (19) @@ -125,20 +125,20 @@ Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(22) CometWindowExec +(22) CometColumnarToRow [codegen id : 2] Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22], [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] -(23) CometProject -Input [8]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] -Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +(23) Window +Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] +Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] -(24) CometTakeOrderedAndProject -Input [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#18 DESC NULLS LAST,CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST,rank_within_parent#22 ASC NULLS FIRST], output=[total_sum#17,i_category#12,i_class#13,lochierarchy#18,rank_within_parent#22]), [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22], 100, 0, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +(24) Project [codegen id : 3] +Output [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +Input [8]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] -(25) CometColumnarToRow [codegen id : 2] +(25) TakeOrderedAndProject Input [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt index 48e02a36c3..7653245f61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS total_sum#3, i_category#4, i_class#5, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS lochierarchy#7, MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS _w0#8, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS _w1#9, CASE WHEN (cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint) = 0) THEN i_category#4 END AS _w2#10)] @@ -30,4 +30,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 28 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt index 98ab180c02..539b436013 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86.native_iceberg_compat/simplified.txt @@ -1,37 +1,39 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] - CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (3) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt index f07df4bf44..de788a42e2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometTakeOrderedAndProject (24) - +- CometProject (23) - +- CometWindowExec (22) +TakeOrderedAndProject (25) ++- * Project (24) + +- Window (23) + +- * CometColumnarToRow (22) +- CometSort (21) +- CometColumnarExchange (20) +- * HashAggregate (19) @@ -125,20 +125,20 @@ Arguments: hashpartitioning(_w1#20, _w2#21, 5), ENSURE_REQUIREMENTS, CometColumn Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21], [_w1#20 ASC NULLS FIRST, _w2#21 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(22) CometWindowExec +(22) CometColumnarToRow [codegen id : 2] Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] -Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22], [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] -(23) CometProject -Input [8]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] -Arguments: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +(23) Window +Input [7]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21] +Arguments: [rank(_w0#19) windowspecdefinition(_w1#20, _w2#21, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#22], [_w1#20, _w2#21], [_w0#19 DESC NULLS LAST] -(24) CometTakeOrderedAndProject -Input [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#18 DESC NULLS LAST,CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST,rank_within_parent#22 ASC NULLS FIRST], output=[total_sum#17,i_category#12,i_class#13,lochierarchy#18,rank_within_parent#22]), [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22], 100, 0, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +(24) Project [codegen id : 3] +Output [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +Input [8]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, _w0#19, _w1#20, _w2#21, rank_within_parent#22] -(25) CometColumnarToRow [codegen id : 2] +(25) TakeOrderedAndProject Input [5]: [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] +Arguments: 100, [lochierarchy#18 DESC NULLS LAST, CASE WHEN (lochierarchy#18 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#22 ASC NULLS FIRST], [total_sum#17, i_category#12, i_class#13, lochierarchy#18, rank_within_parent#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt index 48e02a36c3..7653245f61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS total_sum#3, i_category#4, i_class#5, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS lochierarchy#7, MakeDecimal(sum(UnscaledValue(ws_net_paid#1))#2,17,2) AS _w0#8, (cast((shiftright(spark_grouping_id#6, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint)) AS _w1#9, CASE WHEN (cast((shiftright(spark_grouping_id#6, 0) & 1) as tinyint) = 0) THEN i_category#4 END AS _w2#10)] @@ -30,4 +30,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 28 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt index 98ab180c02..539b436013 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q86/simplified.txt @@ -1,37 +1,39 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometColumnarExchange [_w1,_w2] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] - CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (3) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,spark_grouping_id,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt index 75d2f28ba6..974bebdf5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (30) -+- CometTakeOrderedAndProject (29) - +- CometProject (28) - +- CometFilter (27) - +- CometWindowExec (26) +TakeOrderedAndProject (30) ++- * Project (29) + +- * Filter (28) + +- Window (27) + +- * CometColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_ Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] -(26) CometWindowExec +(26) CometColumnarToRow [codegen id : 2] Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23], [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] -(27) CometFilter +(27) Window +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] +Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] + +(28) Filter [codegen id : 3] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END -(28) CometProject +(29) Project [codegen id : 3] +Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] - -(29) CometTakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST,s_store_name#17 ASC NULLS FIRST], output=[i_category#7,i_class#6,i_brand#5,s_store_name#17,s_company_name#18,d_moy#15,sum_sales#21,avg_monthly_sales#23]), [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23], 100, 0, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -(30) CometColumnarToRow [codegen id : 2] +(30) TakeOrderedAndProject Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] +Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt index 224e191e1d..451a45a044 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_class#2, i_brand#3, s_store_name#4, s_company_name#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt index 5c61e618cb..4fc9029de0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89.native_iceberg_compat/simplified.txt @@ -1,42 +1,44 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - CometProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - CometFilter [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] - CometWindowExec [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] + WholeStageCodegen (3) + Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt index 75d2f28ba6..974bebdf5e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (30) -+- CometTakeOrderedAndProject (29) - +- CometProject (28) - +- CometFilter (27) - +- CometWindowExec (26) +TakeOrderedAndProject (30) ++- * Project (29) + +- * Filter (28) + +- Window (27) + +- * CometColumnarToRow (26) +- CometSort (25) +- CometColumnarExchange (24) +- * HashAggregate (23) @@ -150,24 +150,24 @@ Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_ Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] -(26) CometWindowExec +(26) CometColumnarToRow [codegen id : 2] Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23], [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] -(27) CometFilter +(27) Window +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22] +Arguments: [avg(_w0#22) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#23], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] + +(28) Filter [codegen id : 3] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] Condition : CASE WHEN NOT (avg_monthly_sales#23 = 0.000000) THEN ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000) END -(28) CometProject +(29) Project [codegen id : 3] +Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, _w0#22, avg_monthly_sales#23] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] - -(29) CometTakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST,s_store_name#17 ASC NULLS FIRST], output=[i_category#7,i_class#6,i_brand#5,s_store_name#17,s_company_name#18,d_moy#15,sum_sales#21,avg_monthly_sales#23]), [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23], 100, 0, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] -(30) CometColumnarToRow [codegen id : 2] +(30) TakeOrderedAndProject Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] +Arguments: 100, [(sum_sales#21 - avg_monthly_sales#23) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#21, avg_monthly_sales#23] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt index 224e191e1d..451a45a044 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_class#2, i_brand#3, s_store_name#4, s_company_name#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 30 out of 33 eligible operators (90%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 26 out of 33 eligible operators (78%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt index 5c61e618cb..4fc9029de0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q89/simplified.txt @@ -1,42 +1,44 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - CometProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - CometFilter [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] - CometWindowExec [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] + WholeStageCodegen (3) + Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt index 71782c1e0a..3c2b54e5b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/explain.txt @@ -1,30 +1,31 @@ == Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometExchange (23) - +- CometProject (22) - +- CometWindowExec (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (27) ++- CometProject (26) + +- CometSort (25) + +- CometColumnarExchange (24) + +- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -122,58 +123,61 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject +(22) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(23) Project [codegen id : 3] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(23) CometExchange +(24) CometColumnarExchange Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometSort +(25) CometSort Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(25) CometProject +(26) CometProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(26) CometColumnarToRow [codegen id : 2] +(27) CometColumnarToRow [codegen id : 4] Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) +BroadcastExchange (32) ++- * CometColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (28) -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter +(29) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(29) CometProject +(30) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(31) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(31) BroadcastExchange +(32) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt index c5e17e1883..434f1aa99e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/extended.txt @@ -1,34 +1,35 @@ CometColumnarToRow +- CometProject +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 29 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt index 770e245610..3c91801f00 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98.native_iceberg_compat/simplified.txt @@ -1,38 +1,43 @@ -WholeStageCodegen (2) +WholeStageCodegen (4) CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (3) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt index 71782c1e0a..3c2b54e5b2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/explain.txt @@ -1,30 +1,31 @@ == Physical Plan == -* CometColumnarToRow (26) -+- CometProject (25) - +- CometSort (24) - +- CometExchange (23) - +- CometProject (22) - +- CometWindowExec (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (27) ++- CometProject (26) + +- CometSort (25) + +- CometColumnarExchange (24) + +- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -122,58 +123,61 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject +(22) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(23) Project [codegen id : 3] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, i_item_id#11, _we0#20] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21, i_item_id#11] -(23) CometExchange +(24) CometColumnarExchange Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometSort +(25) CometSort Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(25) CometProject +(26) CometProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -(26) CometColumnarToRow [codegen id : 2] +(27) CometColumnarToRow [codegen id : 4] Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (31) -+- * CometColumnarToRow (30) - +- CometProject (29) - +- CometFilter (28) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) +BroadcastExchange (32) ++- * CometColumnarToRow (31) + +- CometProject (30) + +- CometFilter (29) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (28) -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(28) CometFilter +(29) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(29) CometProject +(30) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(30) CometColumnarToRow [codegen id : 1] +(31) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(31) BroadcastExchange +(32) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt index c5e17e1883..434f1aa99e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/extended.txt @@ -1,34 +1,35 @@ CometColumnarToRow +- CometProject +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_desc#1, i_category#2, i_class#3, i_current_price#4, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS itemrevenue#7, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#6,17,2) AS _w0#8, i_item_id#9)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 29 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 29 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt index 770e245610..3c91801f00 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q98/simplified.txt @@ -1,38 +1,43 @@ -WholeStageCodegen (2) +WholeStageCodegen (4) CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (3) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt index a207bd8844..93308784a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometProject (21) - +- CometWindowExec (20) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] + +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(22) CometTakeOrderedAndProject +(23) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -(23) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt index 7b9553ba1c..9ddd3aeb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt index fbfb52828a..fb83fd2f9a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.native_iceberg_compat/simplified.txt @@ -1,33 +1,35 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt index a207bd8844..93308784a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometProject (21) - +- CometWindowExec (20) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] + +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(22) CometTakeOrderedAndProject +(23) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -(23) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt index 7b9553ba1c..9ddd3aeb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt index fbfb52828a..fb83fd2f9a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/simplified.txt @@ -1,33 +1,35 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt index 1d8693e4aa..c23383e09a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometProject (21) - +- CometWindowExec (20) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] + +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(22) CometTakeOrderedAndProject +(23) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -(23) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt index 3bd3bd46f8..400633ed9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt index 2d72ca58d6..2958d060fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.native_iceberg_compat/simplified.txt @@ -1,33 +1,35 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt index 1d8693e4aa..c23383e09a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometProject (21) - +- CometWindowExec (20) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] + +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(22) CometTakeOrderedAndProject +(23) TakeOrderedAndProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] - -(23) CometColumnarToRow [codegen id : 1] -Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt index 3bd3bd46f8..400633ed9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt index 2d72ca58d6..2958d060fe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/simplified.txt @@ -1,33 +1,35 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt index 97885ed563..ae1d4c242b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (30) -+- CometTakeOrderedAndProject (29) - +- CometProject (28) - +- CometWindowExec (27) +TakeOrderedAndProject (30) ++- * Project (29) + +- Window (28) + +- * CometColumnarToRow (27) +- CometSort (26) +- CometExchange (25) +- CometHashAggregate (24) @@ -153,20 +153,20 @@ Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNative Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] -(27) CometWindowExec +(27) CometColumnarToRow [codegen id : 1] Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26], [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] -(28) CometProject -Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] -Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +(28) Window +Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] -(29) CometTakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#22 DESC NULLS LAST,CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST,rank_within_parent#26 ASC NULLS FIRST], output=[gross_margin#21,i_category#16,i_class#17,lochierarchy#22,rank_within_parent#26]), [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26], 100, 0, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +(29) Project [codegen id : 2] +Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] -(30) CometColumnarToRow [codegen id : 1] +(30) TakeOrderedAndProject Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt index 0f5f924dd6..31fe2cc028 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 32 out of 34 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt index b6a5cf7024..61782e3011 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.native_iceberg_compat/simplified.txt @@ -1,40 +1,42 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] - CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (2) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt index 97885ed563..ae1d4c242b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (30) -+- CometTakeOrderedAndProject (29) - +- CometProject (28) - +- CometWindowExec (27) +TakeOrderedAndProject (30) ++- * Project (29) + +- Window (28) + +- * CometColumnarToRow (27) +- CometSort (26) +- CometExchange (25) +- CometHashAggregate (24) @@ -153,20 +153,20 @@ Arguments: hashpartitioning(_w1#24, _w2#25, 5), ENSURE_REQUIREMENTS, CometNative Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25], [_w1#24 ASC NULLS FIRST, _w2#25 ASC NULLS FIRST, _w0#23 ASC NULLS FIRST] -(27) CometWindowExec +(27) CometColumnarToRow [codegen id : 1] Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] -Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26], [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] -(28) CometProject -Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] -Arguments: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +(28) Window +Input [7]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25] +Arguments: [rank(_w0#23) windowspecdefinition(_w1#24, _w2#25, _w0#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#26], [_w1#24, _w2#25], [_w0#23 ASC NULLS FIRST] -(29) CometTakeOrderedAndProject -Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#22 DESC NULLS LAST,CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST,rank_within_parent#26 ASC NULLS FIRST], output=[gross_margin#21,i_category#16,i_class#17,lochierarchy#22,rank_within_parent#26]), [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26], 100, 0, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +(29) Project [codegen id : 2] +Output [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +Input [8]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, _w0#23, _w1#24, _w2#25, rank_within_parent#26] -(30) CometColumnarToRow [codegen id : 1] +(30) TakeOrderedAndProject Input [5]: [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] +Arguments: 100, [lochierarchy#22 DESC NULLS LAST, CASE WHEN (lochierarchy#22 = 0) THEN i_category#16 END ASC NULLS FIRST, rank_within_parent#26 ASC NULLS FIRST], [gross_margin#21, i_category#16, i_class#17, lochierarchy#22, rank_within_parent#26] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt index 0f5f924dd6..31fe2cc028 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 32 out of 34 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt index b6a5cf7024..61782e3011 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/simplified.txt @@ -1,40 +1,42 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] - CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] - CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #6 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (2) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,spark_grouping_id,sum,sum] + CometExpand [i_category,i_class] [ss_ext_sales_price,ss_net_profit,i_category,i_class,spark_grouping_id] + CometProject [ss_ext_sales_price,ss_net_profit,i_category,i_class] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #6 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt index bb7bf636c2..134ab7939e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/explain.txt @@ -1,36 +1,38 @@ == Physical Plan == -* CometColumnarToRow (32) -+- CometTakeOrderedAndProject (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometProject (27) - : +- CometBroadcastHashJoin (26) - : :- CometProject (21) - : : +- CometSortMergeJoin (20) - : : :- CometSort (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometWindowExec (10) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometSort (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometWindowExec (16) - : : +- CometSort (15) - : : +- ReusedExchange (14) - : +- CometBroadcastExchange (25) - : +- CometProject (24) - : +- CometFilter (23) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - +- ReusedExchange (28) +TakeOrderedAndProject (34) ++- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (23) + : : +- * SortMergeJoin Inner (22) + : : :- * Sort (14) + : : : +- * Project (13) + : : : +- * Filter (12) + : : : +- Window (11) + : : : +- * CometColumnarToRow (10) + : : : +- CometSort (9) + : : : +- CometExchange (8) + : : : +- CometFilter (7) + : : : +- CometHashAggregate (6) + : : : +- CometExchange (5) + : : : +- CometHashAggregate (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- * Sort (21) + : : +- * Project (20) + : : +- * Filter (19) + : : +- Window (18) + : : +- * CometColumnarToRow (17) + : : +- CometSort (16) + : : +- ReusedExchange (15) + : +- BroadcastExchange (28) + : +- * CometColumnarToRow (27) + : +- CometProject (26) + : +- CometFilter (25) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + +- ReusedExchange (31) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -74,143 +76,152 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(10) CometWindowExec +(10) CometColumnarToRow [codegen id : 1] Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8, rnk#11], [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] -(11) CometFilter +(11) Window +Input [2]: [item_sk#7, rank_col#8] +Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] + +(12) Filter [codegen id : 2] Input [3]: [item_sk#7, rank_col#8, rnk#11] Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) -(12) CometProject +(13) Project [codegen id : 2] +Output [2]: [item_sk#7, rnk#11] Input [3]: [item_sk#7, rank_col#8, rnk#11] -Arguments: [item_sk#7, rnk#11], [item_sk#7, rnk#11] -(13) CometSort +(14) Sort [codegen id : 2] Input [2]: [item_sk#7, rnk#11] -Arguments: [item_sk#7, rnk#11], [rnk#11 ASC NULLS FIRST] +Arguments: [rnk#11 ASC NULLS FIRST], false, 0 -(14) ReusedExchange [Reuses operator id: 8] +(15) ReusedExchange [Reuses operator id: 8] Output [2]: [item_sk#12, rank_col#13] -(15) CometSort +(16) CometSort Input [2]: [item_sk#12, rank_col#13] Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] -(16) CometWindowExec +(17) CometColumnarToRow [codegen id : 3] Input [2]: [item_sk#12, rank_col#13] -Arguments: [item_sk#12, rank_col#13, rnk#14], [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] -(17) CometFilter +(18) Window +Input [2]: [item_sk#12, rank_col#13] +Arguments: [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] + +(19) Filter [codegen id : 4] Input [3]: [item_sk#12, rank_col#13, rnk#14] Condition : ((rnk#14 < 11) AND isnotnull(item_sk#12)) -(18) CometProject +(20) Project [codegen id : 4] +Output [2]: [item_sk#12, rnk#14] Input [3]: [item_sk#12, rank_col#13, rnk#14] -Arguments: [item_sk#12, rnk#14], [item_sk#12, rnk#14] -(19) CometSort +(21) Sort [codegen id : 4] Input [2]: [item_sk#12, rnk#14] -Arguments: [item_sk#12, rnk#14], [rnk#14 ASC NULLS FIRST] +Arguments: [rnk#14 ASC NULLS FIRST], false, 0 -(20) CometSortMergeJoin -Left output [2]: [item_sk#7, rnk#11] -Right output [2]: [item_sk#12, rnk#14] -Arguments: [rnk#11], [rnk#14], Inner +(22) SortMergeJoin [codegen id : 7] +Left keys [1]: [rnk#11] +Right keys [1]: [rnk#14] +Join type: Inner +Join condition: None -(21) CometProject +(23) Project [codegen id : 7] +Output [3]: [item_sk#7, rnk#11, item_sk#12] Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] -Arguments: [item_sk#7, rnk#11, item_sk#12], [item_sk#7, rnk#11, item_sk#12] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [2]: [i_item_sk#15, i_product_name#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(25) CometFilter Input [2]: [i_item_sk#15, i_product_name#16] Condition : isnotnull(i_item_sk#15) -(24) CometProject +(26) CometProject Input [2]: [i_item_sk#15, i_product_name#16] Arguments: [i_item_sk#15, i_product_name#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#16, 50, true, false, true) AS i_product_name#17] -(25) CometBroadcastExchange +(27) CometColumnarToRow [codegen id : 5] +Input [2]: [i_item_sk#15, i_product_name#17] + +(28) BroadcastExchange Input [2]: [i_item_sk#15, i_product_name#17] -Arguments: [i_item_sk#15, i_product_name#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) CometBroadcastHashJoin -Left output [3]: [item_sk#7, rnk#11, item_sk#12] -Right output [2]: [i_item_sk#15, i_product_name#17] -Arguments: [item_sk#7], [i_item_sk#15], Inner, BuildRight +(29) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [item_sk#7] +Right keys [1]: [i_item_sk#15] +Join type: Inner +Join condition: None -(27) CometProject +(30) Project [codegen id : 7] +Output [3]: [rnk#11, item_sk#12, i_product_name#17] Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#17] -Arguments: [rnk#11, item_sk#12, i_product_name#17], [rnk#11, item_sk#12, i_product_name#17] -(28) ReusedExchange [Reuses operator id: 25] +(31) ReusedExchange [Reuses operator id: 28] Output [2]: [i_item_sk#18, i_product_name#19] -(29) CometBroadcastHashJoin -Left output [3]: [rnk#11, item_sk#12, i_product_name#17] -Right output [2]: [i_item_sk#18, i_product_name#19] -Arguments: [item_sk#12], [i_item_sk#18], Inner, BuildRight +(32) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [item_sk#12] +Right keys [1]: [i_item_sk#18] +Join type: Inner +Join condition: None -(30) CometProject +(33) Project [codegen id : 7] +Output [3]: [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] Input [5]: [rnk#11, item_sk#12, i_product_name#17, i_item_sk#18, i_product_name#19] -Arguments: [rnk#11, best_performing#20, worst_performing#21], [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] - -(31) CometTakeOrderedAndProject -Input [3]: [rnk#11, best_performing#20, worst_performing#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[rnk#11 ASC NULLS FIRST], output=[rnk#11,best_performing#20,worst_performing#21]), [rnk#11, best_performing#20, worst_performing#21], 100, 0, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_performing#21] -(32) CometColumnarToRow [codegen id : 1] +(34) TakeOrderedAndProject Input [3]: [rnk#11, best_performing#20, worst_performing#21] +Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_performing#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (39) -+- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (33) +* CometColumnarToRow (41) ++- CometHashAggregate (40) + +- CometExchange (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (35) -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(34) CometFilter +(36) CometFilter Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Condition : ((isnotnull(ss_store_sk#23) AND (ss_store_sk#23 = 4)) AND isnull(ss_addr_sk#22)) -(35) CometProject +(37) CometProject Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Arguments: [ss_store_sk#23, ss_net_profit#24], [ss_store_sk#23, ss_net_profit#24] -(36) CometHashAggregate +(38) CometHashAggregate Input [2]: [ss_store_sk#23, ss_net_profit#24] Keys [1]: [ss_store_sk#23] Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#24))] -(37) CometExchange +(39) CometExchange Input [3]: [ss_store_sk#23, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(38) CometHashAggregate +(40) CometHashAggregate Input [3]: [ss_store_sk#23, sum#26, count#27] Keys [1]: [ss_store_sk#23] Functions [1]: [avg(UnscaledValue(ss_net_profit#24))] -(39) CometColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [rank_col#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt index 44ae94ee91..ff284b0f37 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/extended.txt @@ -1,60 +1,63 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometProject - : : : +- CometFilter - : : : +- CometWindowExec - : : : +- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- SortMergeJoin + : : :- Sort + : : : +- Project + : : : +- Filter + : : : +- Window [COMET: Window expressions are not supported] + : : : +- CometColumnarToRow + : : : +- CometSort + : : : +- CometExchange + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- Sort + : : +- Project + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 53 out of 55 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 55 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt index 705ece31b6..e3ec416397 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.native_iceberg_compat/simplified.txt @@ -1,44 +1,59 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [rnk,best_performing,worst_performing] - CometProject [i_product_name,i_product_name] [rnk,best_performing,worst_performing] - CometBroadcastHashJoin [rnk,item_sk,i_product_name,i_item_sk,i_product_name] - CometProject [rnk,item_sk,i_product_name] - CometBroadcastHashJoin [item_sk,rnk,item_sk,i_item_sk,i_product_name] - CometProject [item_sk,rnk,item_sk] - CometSortMergeJoin [item_sk,rnk,item_sk,rnk] - CometSort [item_sk,rnk] - CometProject [item_sk,rnk] - CometFilter [item_sk,rank_col,rnk] - CometWindowExec [item_sk,rank_col,rnk] - CometSort [item_sk,rank_col] - CometExchange #1 - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometSort [item_sk,rnk] - CometProject [item_sk,rnk] - CometFilter [item_sk,rank_col,rnk] - CometWindowExec [item_sk,rank_col,rnk] - CometSort [item_sk,rank_col] - ReusedExchange [item_sk,rank_col] #1 - CometBroadcastExchange [i_item_sk,i_product_name] #4 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] - ReusedExchange [i_item_sk,i_product_name] #4 +TakeOrderedAndProject [rnk,best_performing,worst_performing] + WholeStageCodegen (7) + Project [rnk,i_product_name,i_product_name] + BroadcastHashJoin [item_sk,i_item_sk] + Project [rnk,item_sk,i_product_name] + BroadcastHashJoin [item_sk,i_item_sk] + Project [item_sk,rnk,item_sk] + SortMergeJoin [rnk,rnk] + InputAdapter + WholeStageCodegen (2) + Sort [rnk] + Project [item_sk,rnk] + Filter [rnk,item_sk] + InputAdapter + Window [rank_col] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item_sk,rank_col] + CometExchange #1 + CometFilter [item_sk,rank_col] + Subquery #1 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_store_sk] #3 + CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] + CometProject [ss_store_sk,ss_net_profit] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_item_sk] #2 + CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] + CometProject [ss_item_sk,ss_net_profit] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [rnk] + Project [item_sk,rnk] + Filter [rnk,item_sk] + InputAdapter + Window [rank_col] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [item_sk,rank_col] + ReusedExchange [item_sk,rank_col] #1 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] + InputAdapter + ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt index bb7bf636c2..134ab7939e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt @@ -1,36 +1,38 @@ == Physical Plan == -* CometColumnarToRow (32) -+- CometTakeOrderedAndProject (31) - +- CometProject (30) - +- CometBroadcastHashJoin (29) - :- CometProject (27) - : +- CometBroadcastHashJoin (26) - : :- CometProject (21) - : : +- CometSortMergeJoin (20) - : : :- CometSort (13) - : : : +- CometProject (12) - : : : +- CometFilter (11) - : : : +- CometWindowExec (10) - : : : +- CometSort (9) - : : : +- CometExchange (8) - : : : +- CometFilter (7) - : : : +- CometHashAggregate (6) - : : : +- CometExchange (5) - : : : +- CometHashAggregate (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometSort (19) - : : +- CometProject (18) - : : +- CometFilter (17) - : : +- CometWindowExec (16) - : : +- CometSort (15) - : : +- ReusedExchange (14) - : +- CometBroadcastExchange (25) - : +- CometProject (24) - : +- CometFilter (23) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (22) - +- ReusedExchange (28) +TakeOrderedAndProject (34) ++- * Project (33) + +- * BroadcastHashJoin Inner BuildRight (32) + :- * Project (30) + : +- * BroadcastHashJoin Inner BuildRight (29) + : :- * Project (23) + : : +- * SortMergeJoin Inner (22) + : : :- * Sort (14) + : : : +- * Project (13) + : : : +- * Filter (12) + : : : +- Window (11) + : : : +- * CometColumnarToRow (10) + : : : +- CometSort (9) + : : : +- CometExchange (8) + : : : +- CometFilter (7) + : : : +- CometHashAggregate (6) + : : : +- CometExchange (5) + : : : +- CometHashAggregate (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- * Sort (21) + : : +- * Project (20) + : : +- * Filter (19) + : : +- Window (18) + : : +- * CometColumnarToRow (17) + : : +- CometSort (16) + : : +- ReusedExchange (15) + : +- BroadcastExchange (28) + : +- * CometColumnarToRow (27) + : +- CometProject (26) + : +- CometFilter (25) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + +- ReusedExchange (31) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -74,143 +76,152 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [2]: [item_sk#7, rank_col#8] Arguments: [item_sk#7, rank_col#8], [rank_col#8 ASC NULLS FIRST] -(10) CometWindowExec +(10) CometColumnarToRow [codegen id : 1] Input [2]: [item_sk#7, rank_col#8] -Arguments: [item_sk#7, rank_col#8, rnk#11], [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] -(11) CometFilter +(11) Window +Input [2]: [item_sk#7, rank_col#8] +Arguments: [rank(rank_col#8) windowspecdefinition(rank_col#8 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#11], [rank_col#8 ASC NULLS FIRST] + +(12) Filter [codegen id : 2] Input [3]: [item_sk#7, rank_col#8, rnk#11] Condition : ((rnk#11 < 11) AND isnotnull(item_sk#7)) -(12) CometProject +(13) Project [codegen id : 2] +Output [2]: [item_sk#7, rnk#11] Input [3]: [item_sk#7, rank_col#8, rnk#11] -Arguments: [item_sk#7, rnk#11], [item_sk#7, rnk#11] -(13) CometSort +(14) Sort [codegen id : 2] Input [2]: [item_sk#7, rnk#11] -Arguments: [item_sk#7, rnk#11], [rnk#11 ASC NULLS FIRST] +Arguments: [rnk#11 ASC NULLS FIRST], false, 0 -(14) ReusedExchange [Reuses operator id: 8] +(15) ReusedExchange [Reuses operator id: 8] Output [2]: [item_sk#12, rank_col#13] -(15) CometSort +(16) CometSort Input [2]: [item_sk#12, rank_col#13] Arguments: [item_sk#12, rank_col#13], [rank_col#13 DESC NULLS LAST] -(16) CometWindowExec +(17) CometColumnarToRow [codegen id : 3] Input [2]: [item_sk#12, rank_col#13] -Arguments: [item_sk#12, rank_col#13, rnk#14], [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] -(17) CometFilter +(18) Window +Input [2]: [item_sk#12, rank_col#13] +Arguments: [rank(rank_col#13) windowspecdefinition(rank_col#13 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rnk#14], [rank_col#13 DESC NULLS LAST] + +(19) Filter [codegen id : 4] Input [3]: [item_sk#12, rank_col#13, rnk#14] Condition : ((rnk#14 < 11) AND isnotnull(item_sk#12)) -(18) CometProject +(20) Project [codegen id : 4] +Output [2]: [item_sk#12, rnk#14] Input [3]: [item_sk#12, rank_col#13, rnk#14] -Arguments: [item_sk#12, rnk#14], [item_sk#12, rnk#14] -(19) CometSort +(21) Sort [codegen id : 4] Input [2]: [item_sk#12, rnk#14] -Arguments: [item_sk#12, rnk#14], [rnk#14 ASC NULLS FIRST] +Arguments: [rnk#14 ASC NULLS FIRST], false, 0 -(20) CometSortMergeJoin -Left output [2]: [item_sk#7, rnk#11] -Right output [2]: [item_sk#12, rnk#14] -Arguments: [rnk#11], [rnk#14], Inner +(22) SortMergeJoin [codegen id : 7] +Left keys [1]: [rnk#11] +Right keys [1]: [rnk#14] +Join type: Inner +Join condition: None -(21) CometProject +(23) Project [codegen id : 7] +Output [3]: [item_sk#7, rnk#11, item_sk#12] Input [4]: [item_sk#7, rnk#11, item_sk#12, rnk#14] -Arguments: [item_sk#7, rnk#11, item_sk#12], [item_sk#7, rnk#11, item_sk#12] -(22) CometScan [native_iceberg_compat] parquet spark_catalog.default.item +(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.item Output [2]: [i_item_sk#15, i_product_name#16] Batched: true Location [not included in comparison]/{warehouse_dir}/item] PushedFilters: [IsNotNull(i_item_sk)] ReadSchema: struct -(23) CometFilter +(25) CometFilter Input [2]: [i_item_sk#15, i_product_name#16] Condition : isnotnull(i_item_sk#15) -(24) CometProject +(26) CometProject Input [2]: [i_item_sk#15, i_product_name#16] Arguments: [i_item_sk#15, i_product_name#17], [i_item_sk#15, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, i_product_name#16, 50, true, false, true) AS i_product_name#17] -(25) CometBroadcastExchange +(27) CometColumnarToRow [codegen id : 5] +Input [2]: [i_item_sk#15, i_product_name#17] + +(28) BroadcastExchange Input [2]: [i_item_sk#15, i_product_name#17] -Arguments: [i_item_sk#15, i_product_name#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(26) CometBroadcastHashJoin -Left output [3]: [item_sk#7, rnk#11, item_sk#12] -Right output [2]: [i_item_sk#15, i_product_name#17] -Arguments: [item_sk#7], [i_item_sk#15], Inner, BuildRight +(29) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [item_sk#7] +Right keys [1]: [i_item_sk#15] +Join type: Inner +Join condition: None -(27) CometProject +(30) Project [codegen id : 7] +Output [3]: [rnk#11, item_sk#12, i_product_name#17] Input [5]: [item_sk#7, rnk#11, item_sk#12, i_item_sk#15, i_product_name#17] -Arguments: [rnk#11, item_sk#12, i_product_name#17], [rnk#11, item_sk#12, i_product_name#17] -(28) ReusedExchange [Reuses operator id: 25] +(31) ReusedExchange [Reuses operator id: 28] Output [2]: [i_item_sk#18, i_product_name#19] -(29) CometBroadcastHashJoin -Left output [3]: [rnk#11, item_sk#12, i_product_name#17] -Right output [2]: [i_item_sk#18, i_product_name#19] -Arguments: [item_sk#12], [i_item_sk#18], Inner, BuildRight +(32) BroadcastHashJoin [codegen id : 7] +Left keys [1]: [item_sk#12] +Right keys [1]: [i_item_sk#18] +Join type: Inner +Join condition: None -(30) CometProject +(33) Project [codegen id : 7] +Output [3]: [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] Input [5]: [rnk#11, item_sk#12, i_product_name#17, i_item_sk#18, i_product_name#19] -Arguments: [rnk#11, best_performing#20, worst_performing#21], [rnk#11, i_product_name#17 AS best_performing#20, i_product_name#19 AS worst_performing#21] - -(31) CometTakeOrderedAndProject -Input [3]: [rnk#11, best_performing#20, worst_performing#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[rnk#11 ASC NULLS FIRST], output=[rnk#11,best_performing#20,worst_performing#21]), [rnk#11, best_performing#20, worst_performing#21], 100, 0, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_performing#21] -(32) CometColumnarToRow [codegen id : 1] +(34) TakeOrderedAndProject Input [3]: [rnk#11, best_performing#20, worst_performing#21] +Arguments: 100, [rnk#11 ASC NULLS FIRST], [rnk#11, best_performing#20, worst_performing#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 7 Hosting Expression = Subquery scalar-subquery#9, [id=#10] -* CometColumnarToRow (39) -+- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometProject (35) - +- CometFilter (34) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (33) +* CometColumnarToRow (41) ++- CometHashAggregate (40) + +- CometExchange (39) + +- CometHashAggregate (38) + +- CometProject (37) + +- CometFilter (36) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (35) -(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(35) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Batched: true Location [not included in comparison]/{warehouse_dir}/store_sales] PushedFilters: [IsNotNull(ss_store_sk), EqualTo(ss_store_sk,4), IsNull(ss_addr_sk)] ReadSchema: struct -(34) CometFilter +(36) CometFilter Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Condition : ((isnotnull(ss_store_sk#23) AND (ss_store_sk#23 = 4)) AND isnull(ss_addr_sk#22)) -(35) CometProject +(37) CometProject Input [4]: [ss_addr_sk#22, ss_store_sk#23, ss_net_profit#24, ss_sold_date_sk#25] Arguments: [ss_store_sk#23, ss_net_profit#24], [ss_store_sk#23, ss_net_profit#24] -(36) CometHashAggregate +(38) CometHashAggregate Input [2]: [ss_store_sk#23, ss_net_profit#24] Keys [1]: [ss_store_sk#23] Functions [1]: [partial_avg(UnscaledValue(ss_net_profit#24))] -(37) CometExchange +(39) CometExchange Input [3]: [ss_store_sk#23, sum#26, count#27] -Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(ss_store_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(38) CometHashAggregate +(40) CometHashAggregate Input [3]: [ss_store_sk#23, sum#26, count#27] Keys [1]: [ss_store_sk#23] Functions [1]: [avg(UnscaledValue(ss_net_profit#24))] -(39) CometColumnarToRow [codegen id : 1] +(41) CometColumnarToRow [codegen id : 1] Input [1]: [rank_col#28] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt index 44ae94ee91..ff284b0f37 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/extended.txt @@ -1,60 +1,63 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometSortMergeJoin - : : :- CometSort - : : : +- CometProject - : : : +- CometFilter - : : : +- CometWindowExec - : : : +- CometSort - : : : +- CometExchange - : : : +- CometFilter - : : : : +- Subquery - : : : : +- CometColumnarToRow - : : : : +- CometHashAggregate - : : : : +- CometExchange - : : : : +- CometHashAggregate - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometSort - : : +- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometFilter - : : : +- Subquery - : : : +- CometColumnarToRow - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- SortMergeJoin + : : :- Sort + : : : +- Project + : : : +- Filter + : : : +- Window [COMET: Window expressions are not supported] + : : : +- CometColumnarToRow + : : : +- CometSort + : : : +- CometExchange + : : : +- CometFilter + : : : : +- Subquery + : : : : +- CometColumnarToRow + : : : : +- CometHashAggregate + : : : : +- CometExchange + : : : : +- CometHashAggregate + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- Sort + : : +- Project + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometFilter + : : : +- Subquery + : : : +- CometColumnarToRow + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- BroadcastExchange + : +- CometColumnarToRow + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- BroadcastExchange + +- CometColumnarToRow +- CometProject +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 53 out of 55 eligible operators (96%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 36 out of 55 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt index 705ece31b6..e3ec416397 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/simplified.txt @@ -1,44 +1,59 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [rnk,best_performing,worst_performing] - CometProject [i_product_name,i_product_name] [rnk,best_performing,worst_performing] - CometBroadcastHashJoin [rnk,item_sk,i_product_name,i_item_sk,i_product_name] - CometProject [rnk,item_sk,i_product_name] - CometBroadcastHashJoin [item_sk,rnk,item_sk,i_item_sk,i_product_name] - CometProject [item_sk,rnk,item_sk] - CometSortMergeJoin [item_sk,rnk,item_sk,rnk] - CometSort [item_sk,rnk] - CometProject [item_sk,rnk] - CometFilter [item_sk,rank_col,rnk] - CometWindowExec [item_sk,rank_col,rnk] - CometSort [item_sk,rank_col] - CometExchange #1 - CometFilter [item_sk,rank_col] - Subquery #1 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_store_sk] #3 - CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] - CometProject [ss_store_sk,ss_net_profit] - CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] - CometExchange [ss_item_sk] #2 - CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] - CometProject [ss_item_sk,ss_net_profit] - CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometSort [item_sk,rnk] - CometProject [item_sk,rnk] - CometFilter [item_sk,rank_col,rnk] - CometWindowExec [item_sk,rank_col,rnk] - CometSort [item_sk,rank_col] - ReusedExchange [item_sk,rank_col] #1 - CometBroadcastExchange [i_item_sk,i_product_name] #4 - CometProject [i_product_name] [i_item_sk,i_product_name] - CometFilter [i_item_sk,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] - ReusedExchange [i_item_sk,i_product_name] #4 +TakeOrderedAndProject [rnk,best_performing,worst_performing] + WholeStageCodegen (7) + Project [rnk,i_product_name,i_product_name] + BroadcastHashJoin [item_sk,i_item_sk] + Project [rnk,item_sk,i_product_name] + BroadcastHashJoin [item_sk,i_item_sk] + Project [item_sk,rnk,item_sk] + SortMergeJoin [rnk,rnk] + InputAdapter + WholeStageCodegen (2) + Sort [rnk] + Project [item_sk,rnk] + Filter [rnk,item_sk] + InputAdapter + Window [rank_col] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item_sk,rank_col] + CometExchange #1 + CometFilter [item_sk,rank_col] + Subquery #1 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometHashAggregate [ss_store_sk,sum,count] [rank_col,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_store_sk] #3 + CometHashAggregate [ss_net_profit] [ss_store_sk,sum,count] + CometProject [ss_store_sk,ss_net_profit] + CometFilter [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_addr_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometHashAggregate [sum,count] [item_sk,rank_col,ss_item_sk,avg(UnscaledValue(ss_net_profit))] + CometExchange [ss_item_sk] #2 + CometHashAggregate [ss_net_profit] [ss_item_sk,sum,count] + CometProject [ss_item_sk,ss_net_profit] + CometFilter [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_net_profit,ss_sold_date_sk] + InputAdapter + WholeStageCodegen (4) + Sort [rnk] + Project [item_sk,rnk] + Filter [rnk,item_sk] + InputAdapter + Window [rank_col] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [item_sk,rank_col] + ReusedExchange [item_sk,rank_col] #1 + InputAdapter + BroadcastExchange #4 + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometProject [i_product_name] [i_item_sk,i_product_name] + CometFilter [i_item_sk,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_product_name] + InputAdapter + ReusedExchange [i_item_sk,i_product_name] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt index d3e2a1a2d6..1c039203ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/explain.txt @@ -1,46 +1,52 @@ == Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (28) - : : +- CometFilter (27) - : : +- CometWindowExec (26) - : : +- CometFilter (25) - : : +- CometWindowExec (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometWindowExec (33) - : +- CometSort (32) - : +- CometExchange (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - +- ReusedExchange (38) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -153,107 +159,131 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometWindowExec +(24) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometFilter +(25) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(26) Filter [codegen id : 2] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(26) CometWindowExec +(27) Window Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(27) CometFilter +(28) Filter [codegen id : 7] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(28) CometProject +(29) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(29) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -(30) CometHashAggregate +(31) CometHashAggregate Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] -(31) CometExchange +(32) CometExchange Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) CometSort +(33) CometSort Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(33) CometWindowExec +(34) CometColumnarToRow [codegen id : 3] +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] + +(35) Window Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28], [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(34) CometProject +(36) Project [codegen id : 4] +Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] -Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28], [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -(35) CometBroadcastExchange +(37) BroadcastExchange Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(36) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Right output [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)], Inner, BuildRight +(38) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] +Join type: Inner +Join condition: None -(37) CometProject +(39) Project [codegen id : 7] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -(38) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +(40) ReusedExchange [Reuses operator id: 32] +Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] + +(41) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(42) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] + +(43) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(44) Project [codegen id : 6] +Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] +Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] -(39) CometBroadcastHashJoin -Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Right output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#37 - 1)], Inner, BuildRight +(45) BroadcastExchange +Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(40) CometProject -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#38, sum_sales#36 AS nsum#39] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] +Join type: Inner +Join condition: None -(41) CometTakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,s_store_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,s_store_name#15,s_company_name#16,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#38,nsum#39]), [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(47) Project [codegen id : 7] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -(42) CometColumnarToRow [codegen id : 1] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(48) TakeOrderedAndProject +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(45) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(46) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt index 8c361a8340..11fbf5bd11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/extended.txt @@ -1,76 +1,78 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -100,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt index 657e83912e..19124a9e20 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.native_iceberg_compat/simplified.txt @@ -1,51 +1,71 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] + WholeStageCodegen (7) + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (2) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt index d3e2a1a2d6..1c039203ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt @@ -1,46 +1,52 @@ == Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (28) - : : +- CometFilter (27) - : : +- CometWindowExec (26) - : : +- CometFilter (25) - : : +- CometWindowExec (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometWindowExec (33) - : +- CometSort (32) - : +- CometExchange (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - +- ReusedExchange (38) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -153,107 +159,131 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometWindowExec +(24) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometFilter +(25) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(26) Filter [codegen id : 2] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(26) CometWindowExec +(27) Window Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(27) CometFilter +(28) Filter [codegen id : 7] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(28) CometProject +(29) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(29) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -(30) CometHashAggregate +(31) CometHashAggregate Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] -(31) CometExchange +(32) CometExchange Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) CometSort +(33) CometSort Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(33) CometWindowExec +(34) CometColumnarToRow [codegen id : 3] +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] + +(35) Window Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28], [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(34) CometProject +(36) Project [codegen id : 4] +Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] -Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28], [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -(35) CometBroadcastExchange +(37) BroadcastExchange Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(36) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Right output [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)], Inner, BuildRight +(38) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] +Join type: Inner +Join condition: None -(37) CometProject +(39) Project [codegen id : 7] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -(38) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +(40) ReusedExchange [Reuses operator id: 32] +Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] + +(41) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(42) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] + +(43) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(44) Project [codegen id : 6] +Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] +Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] -(39) CometBroadcastHashJoin -Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Right output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#37 - 1)], Inner, BuildRight +(45) BroadcastExchange +Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(40) CometProject -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#38, sum_sales#36 AS nsum#39] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] +Join type: Inner +Join condition: None -(41) CometTakeOrderedAndProject -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,s_store_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,s_store_name#15,s_company_name#16,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#38,nsum#39]), [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(47) Project [codegen id : 7] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -(42) CometColumnarToRow [codegen id : 1] -Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(48) TakeOrderedAndProject +Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(45) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(46) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt index 8c361a8340..11fbf5bd11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/extended.txt @@ -1,76 +1,78 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -100,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt index 657e83912e..19124a9e20 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/simplified.txt @@ -1,51 +1,71 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_brand,s_company_name,d_year,d_moy,psum,nsum] + WholeStageCodegen (7) + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (2) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt index 9c336d7e3c..338f5f4201 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/explain.txt @@ -1,79 +1,82 @@ == Physical Plan == -* CometColumnarToRow (75) -+- CometTakeOrderedAndProject (74) - +- CometHashAggregate (73) - +- CometExchange (72) - +- CometHashAggregate (71) - +- CometUnion (70) - :- CometProject (25) - : +- CometFilter (24) - : +- CometWindowExec (23) - : +- CometSort (22) - : +- CometWindowExec (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- CometProject (47) - : +- CometFilter (46) - : +- CometWindowExec (45) - : +- CometSort (44) - : +- CometWindowExec (43) - : +- CometSort (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometBroadcastExchange (29) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (35) - +- CometProject (69) - +- CometFilter (68) - +- CometWindowExec (67) - +- CometSort (66) - +- CometWindowExec (65) - +- CometSort (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometProject (56) - : +- CometBroadcastHashJoin (55) - : :- CometBroadcastExchange (51) - : : +- CometProject (50) - : : +- CometFilter (49) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) - +- ReusedExchange (57) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -170,27 +173,30 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) CometSort +(22) Window +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] + +(23) Sort [codegen id : 2] Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 -(23) CometWindowExec +(24) Window Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(24) CometFilter +(25) Filter [codegen id : 3] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(25) CometProject +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -198,97 +204,100 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(28) CometProject +(29) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(29) CometBroadcastExchange +(30) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(32) CometProject +(33) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(33) CometBroadcastHashJoin +(34) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(34) CometProject +(35) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(35) ReusedExchange [Reuses operator id: 13] +(36) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(36) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(37) CometProject +(38) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(38) CometHashAggregate +(39) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(39) CometExchange +(40) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) CometHashAggregate +(41) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(41) CometExchange +(42) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(42) CometSort +(43) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(43) CometWindowExec +(44) CometColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] + +(45) Window Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(44) CometSort +(46) Sort [codegen id : 5] Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 -(45) CometWindowExec +(47) Window Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(46) CometFilter +(48) Filter [codegen id : 6] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(47) CometProject +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -296,156 +305,158 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(49) CometFilter +(51) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(50) CometProject +(52) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(51) CometBroadcastExchange +(53) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(54) CometProject +(56) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(56) CometProject +(58) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(57) ReusedExchange [Reuses operator id: 13] +(59) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(59) CometProject +(61) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(60) CometHashAggregate +(62) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(61) CometExchange +(63) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(62) CometHashAggregate +(64) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(65) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(64) CometSort +(66) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(65) CometWindowExec +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(66) CometSort +(68) Window +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] + +(69) Sort [codegen id : 8] Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 -(67) CometWindowExec +(70) Window Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(68) CometFilter +(71) Filter [codegen id : 9] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(69) CometProject +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(70) CometUnion -Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(73) Union -(71) CometHashAggregate +(74) HashAggregate [codegen id : 10] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(72) CometExchange +(75) CometColumnarExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometHashAggregate +(76) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(74) CometTakeOrderedAndProject +(77) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter +(80) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(78) CometProject +(81) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(79) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(80) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt index 6a8de7fe28..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/extended.txt @@ -1,91 +1,94 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- Filter + +- Window + +- Sort + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt index 4f128e83de..ca80833ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.native_iceberg_compat/simplified.txt @@ -1,87 +1,110 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometUnion [channel,item,return_ratio,return_rank,currency_rank] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt index 9c336d7e3c..338f5f4201 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt @@ -1,79 +1,82 @@ == Physical Plan == -* CometColumnarToRow (75) -+- CometTakeOrderedAndProject (74) - +- CometHashAggregate (73) - +- CometExchange (72) - +- CometHashAggregate (71) - +- CometUnion (70) - :- CometProject (25) - : +- CometFilter (24) - : +- CometWindowExec (23) - : +- CometSort (22) - : +- CometWindowExec (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- CometProject (47) - : +- CometFilter (46) - : +- CometWindowExec (45) - : +- CometSort (44) - : +- CometWindowExec (43) - : +- CometSort (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometBroadcastExchange (29) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (35) - +- CometProject (69) - +- CometFilter (68) - +- CometWindowExec (67) - +- CometSort (66) - +- CometWindowExec (65) - +- CometSort (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometProject (56) - : +- CometBroadcastHashJoin (55) - : :- CometBroadcastExchange (51) - : : +- CometProject (50) - : : +- CometFilter (49) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) - +- ReusedExchange (57) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -170,27 +173,30 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) CometSort +(22) Window +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] + +(23) Sort [codegen id : 2] Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 -(23) CometWindowExec +(24) Window Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(24) CometFilter +(25) Filter [codegen id : 3] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(25) CometProject +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -198,97 +204,100 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(28) CometProject +(29) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(29) CometBroadcastExchange +(30) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(32) CometProject +(33) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(33) CometBroadcastHashJoin +(34) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(34) CometProject +(35) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(35) ReusedExchange [Reuses operator id: 13] +(36) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(36) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(37) CometProject +(38) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(38) CometHashAggregate +(39) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(39) CometExchange +(40) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) CometHashAggregate +(41) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(41) CometExchange +(42) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(42) CometSort +(43) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(43) CometWindowExec +(44) CometColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] + +(45) Window Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(44) CometSort +(46) Sort [codegen id : 5] Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 -(45) CometWindowExec +(47) Window Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(46) CometFilter +(48) Filter [codegen id : 6] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(47) CometProject +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -296,156 +305,158 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(49) CometFilter +(51) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(50) CometProject +(52) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(51) CometBroadcastExchange +(53) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(54) CometProject +(56) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(56) CometProject +(58) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(57) ReusedExchange [Reuses operator id: 13] +(59) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(59) CometProject +(61) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(60) CometHashAggregate +(62) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(61) CometExchange +(63) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(62) CometHashAggregate +(64) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(65) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(64) CometSort +(66) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(65) CometWindowExec +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(66) CometSort +(68) Window +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] + +(69) Sort [codegen id : 8] Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 -(67) CometWindowExec +(70) Window Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(68) CometFilter +(71) Filter [codegen id : 9] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(69) CometProject +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(70) CometUnion -Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(73) Union -(71) CometHashAggregate +(74) HashAggregate [codegen id : 10] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(72) CometExchange +(75) CometColumnarExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometHashAggregate +(76) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(74) CometTakeOrderedAndProject +(77) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter +(80) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(78) CometProject +(81) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(79) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(80) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt index 6a8de7fe28..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/extended.txt @@ -1,91 +1,94 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- Filter + +- Window + +- Sort + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt index 4f128e83de..ca80833ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/simplified.txt @@ -1,87 +1,110 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometUnion [channel,item,return_ratio,return_rank,currency_rank] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt index 4c7d196606..683cc25d24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/explain.txt @@ -1,43 +1,45 @@ == Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometFilter (37) - +- CometWindowExec (36) - +- CometSort (35) - +- CometExchange (34) - +- CometProject (33) - +- CometSortMergeJoin (32) - :- CometSort (17) - : +- CometExchange (16) - : +- CometProject (15) - : +- CometWindowExec (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (31) - +- CometExchange (30) - +- CometProject (29) - +- CometWindowExec (28) - +- CometSort (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometExchange (24) - +- CometHashAggregate (23) - +- CometProject (22) - +- CometBroadcastHashJoin (21) - :- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) - +- ReusedExchange (20) +TakeOrderedAndProject (41) ++- * Filter (40) + +- Window (39) + +- * CometColumnarToRow (38) + +- CometSort (37) + +- CometExchange (36) + +- CometProject (35) + +- CometSortMergeJoin (34) + :- CometSort (18) + : +- CometColumnarExchange (17) + : +- * Project (16) + : +- Window (15) + : +- * CometColumnarToRow (14) + : +- CometSort (13) + : +- CometExchange (12) + : +- CometHashAggregate (11) + : +- CometExchange (10) + : +- CometHashAggregate (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- CometSort (33) + +- CometColumnarExchange (32) + +- * Project (31) + +- Window (30) + +- * CometColumnarToRow (29) + +- CometSort (28) + +- CometExchange (27) + +- CometHashAggregate (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometFilter (20) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) + +- ReusedExchange (21) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -102,23 +104,26 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) CometWindowExec +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11], [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) CometProject +(15) Window +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] + +(16) Project [codegen id : 2] +Output [3]: [item_sk#9, d_date#6, cume_sales#11] Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] -Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9, d_date#6, cume_sales#11] -(16) CometExchange +(17) CometColumnarExchange Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) CometSort +(18) CometSort Input [3]: [item_sk#9, d_date#6, cume_sales#11] Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] @@ -126,124 +131,127 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(19) CometFilter +(20) CometFilter Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Condition : isnotnull(ss_item_sk#12) -(20) ReusedExchange [Reuses operator id: 6] +(21) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#16, d_date#17] -(21) CometBroadcastHashJoin +(22) CometBroadcastHashJoin Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Right output [2]: [d_date_sk#16, d_date#17] Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight -(22) CometProject +(23) CometProject Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] -(23) CometHashAggregate +(24) CometHashAggregate Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] Keys [2]: [ss_item_sk#12, d_date#17] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] -(24) CometExchange +(25) CometExchange Input [3]: [ss_item_sk#12, d_date#17, sum#18] Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(25) CometHashAggregate +(26) CometHashAggregate Input [3]: [ss_item_sk#12, d_date#17, sum#18] Keys [2]: [ss_item_sk#12, d_date#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] -(26) CometExchange +(27) CometExchange Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(27) CometSort +(28) CometSort Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(28) CometWindowExec +(29) CometColumnarToRow [codegen id : 3] +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] + +(30) Window Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21], [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] +Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] -(29) CometProject +(31) Project [codegen id : 4] +Output [3]: [item_sk#19, d_date#17, cume_sales#21] Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] -Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19, d_date#17, cume_sales#21] -(30) CometExchange +(32) CometColumnarExchange Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(31) CometSort +(33) CometSort Input [3]: [item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(32) CometSortMergeJoin +(34) CometSortMergeJoin Left output [3]: [item_sk#9, d_date#6, cume_sales#11] Right output [3]: [item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter -(33) CometProject +(35) CometProject Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] -(34) CometExchange +(36) CometExchange Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(35) CometSort +(37) CometSort Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] -(36) CometWindowExec +(38) CometColumnarToRow [codegen id : 5] Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] -(37) CometFilter -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) +(39) Window +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] -(38) CometTakeOrderedAndProject +(40) Filter [codegen id : 6] Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#22 ASC NULLS FIRST,d_date#23 ASC NULLS FIRST], output=[item_sk#22,d_date#23,web_sales#24,store_sales#25,web_cumulative#26,store_cumulative#27]), [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], 100, 0, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) -(39) CometColumnarToRow [codegen id : 1] +(41) TakeOrderedAndProject Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter +(43) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(42) CometProject +(44) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(43) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(44) BroadcastExchange +(46) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt index 3c719aa579..da1ac9c0a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/extended.txt @@ -1,51 +1,53 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 44 out of 47 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt index 9215d1d12f..a5af2e114b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51.native_iceberg_compat/simplified.txt @@ -1,50 +1,62 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometWindowExec [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #2 - CometProject [item_sk,d_date,cume_sales] - CometWindowExec [item_sk,d_date,_w0,ws_item_sk,cume_sales] - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #7 - CometProject [item_sk,d_date,cume_sales] - CometWindowExec [item_sk,d_date,_w0,ss_item_sk,cume_sales] - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #6 +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (6) + Filter [web_cumulative,store_cumulative] + InputAdapter + Window [web_sales,item_sk,d_date,store_sales] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (2) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometExchange [ws_item_sk] #3 + CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #4 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #6 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #7 + WholeStageCodegen (4) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometExchange [ss_item_sk] #8 + CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #9 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt index 4c7d196606..683cc25d24 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/explain.txt @@ -1,43 +1,45 @@ == Physical Plan == -* CometColumnarToRow (39) -+- CometTakeOrderedAndProject (38) - +- CometFilter (37) - +- CometWindowExec (36) - +- CometSort (35) - +- CometExchange (34) - +- CometProject (33) - +- CometSortMergeJoin (32) - :- CometSort (17) - : +- CometExchange (16) - : +- CometProject (15) - : +- CometWindowExec (14) - : +- CometSort (13) - : +- CometExchange (12) - : +- CometHashAggregate (11) - : +- CometExchange (10) - : +- CometHashAggregate (9) - : +- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometSort (31) - +- CometExchange (30) - +- CometProject (29) - +- CometWindowExec (28) - +- CometSort (27) - +- CometExchange (26) - +- CometHashAggregate (25) - +- CometExchange (24) - +- CometHashAggregate (23) - +- CometProject (22) - +- CometBroadcastHashJoin (21) - :- CometFilter (19) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (18) - +- ReusedExchange (20) +TakeOrderedAndProject (41) ++- * Filter (40) + +- Window (39) + +- * CometColumnarToRow (38) + +- CometSort (37) + +- CometExchange (36) + +- CometProject (35) + +- CometSortMergeJoin (34) + :- CometSort (18) + : +- CometColumnarExchange (17) + : +- * Project (16) + : +- Window (15) + : +- * CometColumnarToRow (14) + : +- CometSort (13) + : +- CometExchange (12) + : +- CometHashAggregate (11) + : +- CometExchange (10) + : +- CometHashAggregate (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- CometSort (33) + +- CometColumnarExchange (32) + +- * Project (31) + +- Window (30) + +- * CometColumnarToRow (29) + +- CometSort (28) + +- CometExchange (27) + +- CometHashAggregate (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometFilter (20) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (19) + +- ReusedExchange (21) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -102,23 +104,26 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) CometWindowExec +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11], [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) CometProject +(15) Window +Input [4]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1] +Arguments: [sum(_w0#10) windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] + +(16) Project [codegen id : 2] +Output [3]: [item_sk#9, d_date#6, cume_sales#11] Input [5]: [item_sk#9, d_date#6, _w0#10, ws_item_sk#1, cume_sales#11] -Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9, d_date#6, cume_sales#11] -(16) CometExchange +(17) CometColumnarExchange Input [3]: [item_sk#9, d_date#6, cume_sales#11] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(17) CometSort +(18) CometSort Input [3]: [item_sk#9, d_date#6, cume_sales#11] Arguments: [item_sk#9, d_date#6, cume_sales#11], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(18) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(19) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Batched: true Location: InMemoryFileIndex [] @@ -126,124 +131,127 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(19) CometFilter +(20) CometFilter Input [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Condition : isnotnull(ss_item_sk#12) -(20) ReusedExchange [Reuses operator id: 6] +(21) ReusedExchange [Reuses operator id: 6] Output [2]: [d_date_sk#16, d_date#17] -(21) CometBroadcastHashJoin +(22) CometBroadcastHashJoin Left output [3]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14] Right output [2]: [d_date_sk#16, d_date#17] Arguments: [ss_sold_date_sk#14], [d_date_sk#16], Inner, BuildRight -(22) CometProject +(23) CometProject Input [5]: [ss_item_sk#12, ss_sales_price#13, ss_sold_date_sk#14, d_date_sk#16, d_date#17] Arguments: [ss_item_sk#12, ss_sales_price#13, d_date#17], [ss_item_sk#12, ss_sales_price#13, d_date#17] -(23) CometHashAggregate +(24) CometHashAggregate Input [3]: [ss_item_sk#12, ss_sales_price#13, d_date#17] Keys [2]: [ss_item_sk#12, d_date#17] Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#13))] -(24) CometExchange +(25) CometExchange Input [3]: [ss_item_sk#12, d_date#17, sum#18] Arguments: hashpartitioning(ss_item_sk#12, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(25) CometHashAggregate +(26) CometHashAggregate Input [3]: [ss_item_sk#12, d_date#17, sum#18] Keys [2]: [ss_item_sk#12, d_date#17] Functions [1]: [sum(UnscaledValue(ss_sales_price#13))] -(26) CometExchange +(27) CometExchange Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] Arguments: hashpartitioning(ss_item_sk#12, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(27) CometSort +(28) CometSort Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12], [ss_item_sk#12 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(28) CometWindowExec +(29) CometColumnarToRow [codegen id : 3] +Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] + +(30) Window Input [4]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12] -Arguments: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21], [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] +Arguments: [sum(_w0#20) windowspecdefinition(ss_item_sk#12, d_date#17 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS cume_sales#21], [ss_item_sk#12], [d_date#17 ASC NULLS FIRST] -(29) CometProject +(31) Project [codegen id : 4] +Output [3]: [item_sk#19, d_date#17, cume_sales#21] Input [5]: [item_sk#19, d_date#17, _w0#20, ss_item_sk#12, cume_sales#21] -Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19, d_date#17, cume_sales#21] -(30) CometExchange +(32) CometColumnarExchange Input [3]: [item_sk#19, d_date#17, cume_sales#21] -Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +Arguments: hashpartitioning(item_sk#19, d_date#17, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(31) CometSort +(33) CometSort Input [3]: [item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#19, d_date#17, cume_sales#21], [item_sk#19 ASC NULLS FIRST, d_date#17 ASC NULLS FIRST] -(32) CometSortMergeJoin +(34) CometSortMergeJoin Left output [3]: [item_sk#9, d_date#6, cume_sales#11] Right output [3]: [item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#9, d_date#6], [item_sk#19, d_date#17], FullOuter -(33) CometProject +(35) CometProject Input [6]: [item_sk#9, d_date#6, cume_sales#11, item_sk#19, d_date#17, cume_sales#21] Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#19 END AS item_sk#22, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#17 END AS d_date#23, cume_sales#11 AS web_sales#24, cume_sales#21 AS store_sales#25] -(34) CometExchange +(36) CometExchange Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] Arguments: hashpartitioning(item_sk#22, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(35) CometSort +(37) CometSort Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25], [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST] -(36) CometWindowExec +(38) CometColumnarToRow [codegen id : 5] Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] -Arguments: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] -(37) CometFilter -Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) +(39) Window +Input [4]: [item_sk#22, d_date#23, web_sales#24, store_sales#25] +Arguments: [max(web_sales#24) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS web_cumulative#26, max(store_sales#25) windowspecdefinition(item_sk#22, d_date#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS store_cumulative#27], [item_sk#22], [d_date#23 ASC NULLS FIRST] -(38) CometTakeOrderedAndProject +(40) Filter [codegen id : 6] Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#22 ASC NULLS FIRST,d_date#23 ASC NULLS FIRST], output=[item_sk#22,d_date#23,web_sales#24,store_sales#25,web_cumulative#26,store_cumulative#27]), [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27], 100, 0, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Condition : ((isnotnull(web_cumulative#26) AND isnotnull(store_cumulative#27)) AND (web_cumulative#26 > store_cumulative#27)) -(39) CometColumnarToRow [codegen id : 1] +(41) TakeOrderedAndProject Input [6]: [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] +Arguments: 100, [item_sk#22 ASC NULLS FIRST, d_date#23 ASC NULLS FIRST], [item_sk#22, d_date#23, web_sales#24, store_sales#25, web_cumulative#26, store_cumulative#27] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (44) -+- * CometColumnarToRow (43) - +- CometProject (42) - +- CometFilter (41) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (40) +BroadcastExchange (46) ++- * CometColumnarToRow (45) + +- CometProject (44) + +- CometFilter (43) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (42) -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(41) CometFilter +(43) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1200)) AND (d_month_seq#7 <= 1211)) AND isnotnull(d_date_sk#5)) -(42) CometProject +(44) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(43) CometColumnarToRow [codegen id : 1] +(45) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(44) BroadcastExchange +(46) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 18 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 19 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt index 3c719aa579..da1ac9c0a3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/extended.txt @@ -1,51 +1,53 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometProject +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 44 out of 47 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 37 out of 47 eligible operators (78%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt index 9215d1d12f..a5af2e114b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q51/simplified.txt @@ -1,50 +1,62 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometWindowExec [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #2 - CometProject [item_sk,d_date,cume_sales] - CometWindowExec [item_sk,d_date,_w0,ws_item_sk,cume_sales] - CometSort [item_sk,d_date,_w0,ws_item_sk] - CometExchange [ws_item_sk] #3 - CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #4 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #6 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #7 - CometProject [item_sk,d_date,cume_sales] - CometWindowExec [item_sk,d_date,_w0,ss_item_sk,cume_sales] - CometSort [item_sk,d_date,_w0,ss_item_sk] - CometExchange [ss_item_sk] #8 - CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #9 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #6 +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (6) + Filter [web_cumulative,store_cumulative] + InputAdapter + Window [web_sales,item_sk,d_date,store_sales] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (2) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ws_item_sk,d_date] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ws_item_sk] + CometExchange [ws_item_sk] #3 + CometHashAggregate [sum] [item_sk,d_date,_w0,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #4 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #6 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #7 + WholeStageCodegen (4) + Project [item_sk,d_date,cume_sales] + InputAdapter + Window [_w0,ss_item_sk,d_date] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,_w0,ss_item_sk] + CometExchange [ss_item_sk] #8 + CometHashAggregate [sum] [item_sk,d_date,_w0,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #9 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt index 0dfd5e81c4..2fc38bf838 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometProject (27) - +- CometFilter (26) - +- CometWindowExec (25) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativ Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22], [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] -(26) CometFilter +(26) Window +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] + +(27) Filter [codegen id : 2] Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END -(27) CometProject +(28) Project [codegen id : 2] +Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -Arguments: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] - -(28) CometTakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[avg_quarterly_sales#22 ASC NULLS FIRST,sum_sales#20 ASC NULLS FIRST,i_manufact_id#5 ASC NULLS FIRST], output=[i_manufact_id#5,sum_sales#20,avg_quarterly_sales#22]), [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22], 100, 0, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -(29) CometColumnarToRow [codegen id : 1] +(29) TakeOrderedAndProject Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt index 8adffb63e3..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt index 0a2d088fd8..0c123eb728 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.native_iceberg_compat/simplified.txt @@ -1,39 +1,41 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manufact_id,sum_sales,avg_quarterly_sales] - CometProject [i_manufact_id,sum_sales,avg_quarterly_sales] - CometFilter [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] - CometWindowExec [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] - CometSort [i_manufact_id,sum_sales,_w0] - CometExchange [i_manufact_id] #1 - CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] + WholeStageCodegen (2) + Project [i_manufact_id,sum_sales,avg_quarterly_sales] + Filter [avg_quarterly_sales,sum_sales] + InputAdapter + Window [_w0,i_manufact_id] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_manufact_id,sum_sales,_w0] + CometExchange [i_manufact_id] #1 + CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt index 0dfd5e81c4..2fc38bf838 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometProject (27) - +- CometFilter (26) - +- CometWindowExec (25) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_manufact_id#5, 5), ENSURE_REQUIREMENTS, CometNativ Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] Arguments: [i_manufact_id#5, sum_sales#20, _w0#21], [i_manufact_id#5 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] -Arguments: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22], [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] -(26) CometFilter +(26) Window +Input [3]: [i_manufact_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manufact_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_quarterly_sales#22], [i_manufact_id#5] + +(27) Filter [codegen id : 2] Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] Condition : CASE WHEN (avg_quarterly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_quarterly_sales#22)) / avg_quarterly_sales#22) > 0.1000000000000000) ELSE false END -(27) CometProject +(28) Project [codegen id : 2] +Output [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] Input [4]: [i_manufact_id#5, sum_sales#20, _w0#21, avg_quarterly_sales#22] -Arguments: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] - -(28) CometTakeOrderedAndProject -Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[avg_quarterly_sales#22 ASC NULLS FIRST,sum_sales#20 ASC NULLS FIRST,i_manufact_id#5 ASC NULLS FIRST], output=[i_manufact_id#5,sum_sales#20,avg_quarterly_sales#22]), [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22], 100, 0, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] -(29) CometColumnarToRow [codegen id : 1] +(29) TakeOrderedAndProject Input [3]: [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] +Arguments: 100, [avg_quarterly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST, i_manufact_id#5 ASC NULLS FIRST], [i_manufact_id#5, sum_sales#20, avg_quarterly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt index 8adffb63e3..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt index 0a2d088fd8..0c123eb728 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/simplified.txt @@ -1,39 +1,41 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manufact_id,sum_sales,avg_quarterly_sales] - CometProject [i_manufact_id,sum_sales,avg_quarterly_sales] - CometFilter [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] - CometWindowExec [i_manufact_id,sum_sales,_w0,avg_quarterly_sales] - CometSort [i_manufact_id,sum_sales,_w0] - CometExchange [i_manufact_id] #1 - CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manufact_id,d_qoy] #2 - CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] - CometProject [i_manufact_id,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] - CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] - CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manufact_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [d_date_sk,d_qoy] #5 - CometProject [d_date_sk,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +TakeOrderedAndProject [avg_quarterly_sales,sum_sales,i_manufact_id] + WholeStageCodegen (2) + Project [i_manufact_id,sum_sales,avg_quarterly_sales] + Filter [avg_quarterly_sales,sum_sales] + InputAdapter + Window [_w0,i_manufact_id] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_manufact_id,sum_sales,_w0] + CometExchange [i_manufact_id] #1 + CometHashAggregate [d_qoy,sum] [i_manufact_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manufact_id,d_qoy] #2 + CometHashAggregate [ss_sales_price] [i_manufact_id,d_qoy,sum] + CometProject [i_manufact_id,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy,s_store_sk] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,d_qoy] + CometBroadcastHashJoin [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_qoy] + CometProject [i_manufact_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manufact_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manufact_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manufact_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [d_date_sk,d_qoy] #5 + CometProject [d_date_sk,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_qoy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt index 9d0b44c8c2..eb21a3abd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/explain.txt @@ -1,46 +1,52 @@ == Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (28) - : : +- CometFilter (27) - : : +- CometWindowExec (26) - : : +- CometFilter (25) - : : +- CometWindowExec (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometWindowExec (33) - : +- CometSort (32) - : +- CometExchange (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - +- ReusedExchange (38) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -153,107 +159,131 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometWindowExec +(24) CometColumnarToRow [codegen id : 1] Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometFilter +(25) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(26) Filter [codegen id : 2] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(26) CometWindowExec +(27) Window Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20], [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(27) CometFilter +(28) Filter [codegen id : 7] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(28) CometProject +(29) Project [codegen id : 7] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -(29) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -(30) CometHashAggregate +(31) CometHashAggregate Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -(31) CometExchange +(32) CometExchange Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) CometSort +(33) CometSort Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(33) CometWindowExec +(34) CometColumnarToRow [codegen id : 3] +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] + +(35) Window Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26], [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(34) CometProject +(36) Project [codegen id : 4] +Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] -Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26], [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -(35) CometBroadcastExchange +(37) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(36) CometBroadcastHashJoin -Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Right output [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)], Inner, BuildRight +(38) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] +Join type: Inner +Join condition: None -(37) CometProject +(39) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -(38) ReusedExchange [Reuses operator id: 35] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +(40) ReusedExchange [Reuses operator id: 32] +Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] + +(41) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] + +(42) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] + +(43) Window +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] + +(44) Project [codegen id : 6] +Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] +Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] -(39) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Right output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#30, i_brand#31, cc_name#32, (rn#34 - 1)], Inner, BuildRight +(45) BroadcastExchange +Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(40) CometProject -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#35, sum_sales#33 AS nsum#36] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] +Join type: Inner +Join condition: None -(41) CometTakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST,cc_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,cc_name#15,d_year#12,d_moy#13,avg_monthly_sales#20,sum_sales#17,psum#35,nsum#36]), [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], 100, 0, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +(47) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -(42) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +(48) TakeOrderedAndProject +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(45) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(46) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt index 00de5f32a1..7bee0e61e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/extended.txt @@ -1,76 +1,78 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -100,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt index c0f7f57f6d..b854e818db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.native_iceberg_compat/simplified.txt @@ -1,51 +1,71 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,cc_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 +TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] + WholeStageCodegen (7) + Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (2) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,cc_name] #8 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt index 9d0b44c8c2..eb21a3abd7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt @@ -1,46 +1,52 @@ == Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (28) - : : +- CometFilter (27) - : : +- CometWindowExec (26) - : : +- CometFilter (25) - : : +- CometWindowExec (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometWindowExec (33) - : +- CometSort (32) - : +- CometExchange (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - +- ReusedExchange (38) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -153,107 +159,131 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometWindowExec +(24) CometColumnarToRow [codegen id : 1] Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometFilter +(25) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(26) Filter [codegen id : 2] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(26) CometWindowExec +(27) Window Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20], [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(27) CometFilter +(28) Filter [codegen id : 7] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(28) CometProject +(29) Project [codegen id : 7] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -(29) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -(30) CometHashAggregate +(31) CometHashAggregate Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -(31) CometExchange +(32) CometExchange Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) CometSort +(33) CometSort Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(33) CometWindowExec +(34) CometColumnarToRow [codegen id : 3] +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] + +(35) Window Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26], [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(34) CometProject +(36) Project [codegen id : 4] +Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] -Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26], [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -(35) CometBroadcastExchange +(37) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(36) CometBroadcastHashJoin -Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Right output [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)], Inner, BuildRight +(38) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] +Join type: Inner +Join condition: None -(37) CometProject +(39) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -(38) ReusedExchange [Reuses operator id: 35] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +(40) ReusedExchange [Reuses operator id: 32] +Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] + +(41) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] + +(42) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] + +(43) Window +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] + +(44) Project [codegen id : 6] +Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] +Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] -(39) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Right output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#30, i_brand#31, cc_name#32, (rn#34 - 1)], Inner, BuildRight +(45) BroadcastExchange +Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(40) CometProject -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#35, sum_sales#33 AS nsum#36] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] +Join type: Inner +Join condition: None -(41) CometTakeOrderedAndProject -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST,cc_name#15 ASC NULLS FIRST], output=[i_category#5,i_brand#4,cc_name#15,d_year#12,d_moy#13,avg_monthly_sales#20,sum_sales#17,psum#35,nsum#36]), [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], 100, 0, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +(47) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -(42) CometColumnarToRow [codegen id : 1] -Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +(48) TakeOrderedAndProject +Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(45) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(46) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt index 00de5f32a1..7bee0e61e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/extended.txt @@ -1,76 +1,78 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -100,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt index c0f7f57f6d..b854e818db 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/simplified.txt @@ -1,51 +1,71 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,cc_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 +TakeOrderedAndProject [sum_sales,avg_monthly_sales,cc_name,i_category,i_brand,d_year,d_moy,psum,nsum] + WholeStageCodegen (7) + Project [i_category,i_brand,cc_name,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (2) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,cc_name] #8 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt index aa9e4d43d0..e2a0cceb1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometProject (27) - +- CometFilter (26) - +- CometWindowExec (25) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNative Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22], [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] -(26) CometFilter +(26) Window +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] + +(27) Filter [codegen id : 2] Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END -(27) CometProject +(28) Project [codegen id : 2] +Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -Arguments: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] - -(28) CometTakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_manager_id#5 ASC NULLS FIRST,avg_monthly_sales#22 ASC NULLS FIRST,sum_sales#20 ASC NULLS FIRST], output=[i_manager_id#5,sum_sales#20,avg_monthly_sales#22]), [i_manager_id#5, sum_sales#20, avg_monthly_sales#22], 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -(29) CometColumnarToRow [codegen id : 1] +(29) TakeOrderedAndProject Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt index 8adffb63e3..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt index acdc4ee744..02166879a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.native_iceberg_compat/simplified.txt @@ -1,39 +1,41 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manager_id,sum_sales,avg_monthly_sales] - CometProject [i_manager_id,sum_sales,avg_monthly_sales] - CometFilter [i_manager_id,sum_sales,_w0,avg_monthly_sales] - CometWindowExec [i_manager_id,sum_sales,_w0,avg_monthly_sales] - CometSort [i_manager_id,sum_sales,_w0] - CometExchange [i_manager_id] #1 - CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] + WholeStageCodegen (2) + Project [i_manager_id,sum_sales,avg_monthly_sales] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_manager_id] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_manager_id,sum_sales,_w0] + CometExchange [i_manager_id] #1 + CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manager_id,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt index aa9e4d43d0..e2a0cceb1a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometProject (27) - +- CometFilter (26) - +- CometWindowExec (25) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_manager_id#5, 5), ENSURE_REQUIREMENTS, CometNative Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] Arguments: [i_manager_id#5, sum_sales#20, _w0#21], [i_manager_id#5 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 1] Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] -Arguments: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22], [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] -(26) CometFilter +(26) Window +Input [3]: [i_manager_id#5, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_manager_id#5, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_manager_id#5] + +(27) Filter [codegen id : 2] Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] Condition : CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) ELSE false END -(27) CometProject +(28) Project [codegen id : 2] +Output [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] Input [4]: [i_manager_id#5, sum_sales#20, _w0#21, avg_monthly_sales#22] -Arguments: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] - -(28) CometTakeOrderedAndProject -Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_manager_id#5 ASC NULLS FIRST,avg_monthly_sales#22 ASC NULLS FIRST,sum_sales#20 ASC NULLS FIRST], output=[i_manager_id#5,sum_sales#20,avg_monthly_sales#22]), [i_manager_id#5, sum_sales#20, avg_monthly_sales#22], 100, 0, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] -(29) CometColumnarToRow [codegen id : 1] +(29) TakeOrderedAndProject Input [3]: [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] +Arguments: 100, [i_manager_id#5 ASC NULLS FIRST, avg_monthly_sales#22 ASC NULLS FIRST, sum_sales#20 ASC NULLS FIRST], [i_manager_id#5, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt index 8adffb63e3..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt index acdc4ee744..02166879a9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/simplified.txt @@ -1,39 +1,41 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_manager_id,sum_sales,avg_monthly_sales] - CometProject [i_manager_id,sum_sales,avg_monthly_sales] - CometFilter [i_manager_id,sum_sales,_w0,avg_monthly_sales] - CometWindowExec [i_manager_id,sum_sales,_w0,avg_monthly_sales] - CometSort [i_manager_id,sum_sales,_w0] - CometExchange [i_manager_id] #1 - CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_manager_id,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] - CometProject [i_manager_id,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_item_sk,i_manager_id] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_month_seq,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] - CometBroadcastExchange [s_store_sk] #6 - CometFilter [s_store_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] +TakeOrderedAndProject [i_manager_id,avg_monthly_sales,sum_sales] + WholeStageCodegen (2) + Project [i_manager_id,sum_sales,avg_monthly_sales] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_manager_id] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_manager_id,sum_sales,_w0] + CometExchange [i_manager_id] #1 + CometHashAggregate [d_moy,sum] [i_manager_id,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_manager_id,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_manager_id,d_moy,sum] + CometProject [i_manager_id,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,d_moy,s_store_sk] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_manager_id,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_manager_id,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_item_sk,i_manager_id] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_manager_id] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_month_seq,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_moy] + CometBroadcastExchange [s_store_sk] #6 + CometFilter [s_store_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt index 151d5e1afa..df94b5d986 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (30) -+- CometTakeOrderedAndProject (29) - +- CometFilter (28) - +- CometWindowExec (27) +TakeOrderedAndProject (30) ++- * Filter (29) + +- Window (28) + +- * CometColumnarToRow (27) +- CometSort (26) +- CometExchange (25) +- CometHashAggregate (24) @@ -153,20 +153,20 @@ Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometNativeS Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] -(27) CometWindowExec +(27) CometColumnarToRow [codegen id : 1] Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36], [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] -(28) CometFilter -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Condition : (rk#36 <= 100) +(28) Window +Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] +Arguments: [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] -(29) CometTakeOrderedAndProject +(29) Filter [codegen id : 2] Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#24 ASC NULLS FIRST,i_class#25 ASC NULLS FIRST,i_brand#26 ASC NULLS FIRST,i_product_name#27 ASC NULLS FIRST,d_year#28 ASC NULLS FIRST,d_qoy#29 ASC NULLS FIRST,d_moy#30 ASC NULLS FIRST,s_store_id#31 ASC NULLS FIRST,sumsales#35 ASC NULLS FIRST,rk#36 ASC NULLS FIRST], output=[i_category#24,i_class#25,i_brand#26,i_product_name#27,d_year#28,d_qoy#29,d_moy#30,s_store_id#31,sumsales#35,rk#36]), [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36], 100, 0, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] +Condition : (rk#36 <= 100) -(30) CometColumnarToRow [codegen id : 1] +(30) TakeOrderedAndProject Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] +Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt index da844f8a5a..2d55d60ddc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 32 out of 34 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt index f6ddea4c08..e85c243077 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.native_iceberg_compat/simplified.txt @@ -1,40 +1,42 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometFilter [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometWindowExec [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] - CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] - CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + WholeStageCodegen (2) + Filter [rk] + InputAdapter + Window [sumsales,i_category] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometExchange [i_category] #1 + CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] + CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] + CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt index 151d5e1afa..df94b5d986 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (30) -+- CometTakeOrderedAndProject (29) - +- CometFilter (28) - +- CometWindowExec (27) +TakeOrderedAndProject (30) ++- * Filter (29) + +- Window (28) + +- * CometColumnarToRow (27) +- CometSort (26) +- CometExchange (25) +- CometHashAggregate (24) @@ -153,20 +153,20 @@ Arguments: hashpartitioning(i_category#24, 5), ENSURE_REQUIREMENTS, CometNativeS Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35], [i_category#24 ASC NULLS FIRST, sumsales#35 DESC NULLS LAST] -(27) CometWindowExec +(27) CometColumnarToRow [codegen id : 1] Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] -Arguments: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36], [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] -(28) CometFilter -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Condition : (rk#36 <= 100) +(28) Window +Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35] +Arguments: [rank(sumsales#35) windowspecdefinition(i_category#24, sumsales#35 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [i_category#24], [sumsales#35 DESC NULLS LAST] -(29) CometTakeOrderedAndProject +(29) Filter [codegen id : 2] Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#24 ASC NULLS FIRST,i_class#25 ASC NULLS FIRST,i_brand#26 ASC NULLS FIRST,i_product_name#27 ASC NULLS FIRST,d_year#28 ASC NULLS FIRST,d_qoy#29 ASC NULLS FIRST,d_moy#30 ASC NULLS FIRST,s_store_id#31 ASC NULLS FIRST,sumsales#35 ASC NULLS FIRST,rk#36 ASC NULLS FIRST], output=[i_category#24,i_class#25,i_brand#26,i_product_name#27,d_year#28,d_qoy#29,d_moy#30,s_store_id#31,sumsales#35,rk#36]), [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36], 100, 0, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] +Condition : (rk#36 <= 100) -(30) CometColumnarToRow [codegen id : 1] +(30) TakeOrderedAndProject Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] +Arguments: 100, [i_category#24 ASC NULLS FIRST, i_class#25 ASC NULLS FIRST, i_brand#26 ASC NULLS FIRST, i_product_name#27 ASC NULLS FIRST, d_year#28 ASC NULLS FIRST, d_qoy#29 ASC NULLS FIRST, d_moy#30 ASC NULLS FIRST, s_store_id#31 ASC NULLS FIRST, sumsales#35 ASC NULLS FIRST, rk#36 ASC NULLS FIRST], [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sumsales#35, rk#36] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt index da844f8a5a..2d55d60ddc 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -35,4 +35,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 32 out of 34 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 29 out of 34 eligible operators (85%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt index f6ddea4c08..e85c243077 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt @@ -1,40 +1,42 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometFilter [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometWindowExec [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] - CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] - CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + WholeStageCodegen (2) + Filter [rk] + InputAdapter + Window [sumsales,i_category] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometExchange [i_category] #1 + CometHashAggregate [spark_grouping_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2 + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] + CometExpand [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] + CometProject [ss_quantity,ss_sales_price,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt index d059dabb26..d3610af0f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/explain.txt @@ -1,48 +1,52 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometProject (42) - +- CometWindowExec (41) - +- CometSort (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometExpand (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometWindowExec (26) - +- CometSort (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometFilter (12) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - +- ReusedExchange (19) +TakeOrderedAndProject (48) ++- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) + +- CometSort (44) + +- CometColumnarExchange (43) + +- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * Expand (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * CometColumnarToRow (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- BroadcastExchange (35) + +- * Project (34) + +- * BroadcastHashJoin LeftSemi BuildRight (33) + :- * CometColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) + +- BroadcastExchange (32) + +- * Project (31) + +- * Filter (30) + +- Window (29) + +- * CometColumnarToRow (28) + +- CometSort (27) + +- CometHashAggregate (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometFilter (14) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) + +- ReusedExchange (21) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -85,18 +89,24 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(9) CometColumnarToRow [codegen id : 4] +Input [2]: [ss_store_sk#1, ss_net_profit#2] + +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(10) CometFilter +(11) CometFilter Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(12) CometColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] @@ -104,179 +114,191 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(12) CometFilter +(14) CometFilter Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Condition : isnotnull(ss_store_sk#10) -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [2]: [s_store_sk#7, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter +(16) CometFilter Input [2]: [s_store_sk#7, s_state#9] Condition : isnotnull(s_store_sk#7) -(15) CometProject +(17) CometProject Input [2]: [s_store_sk#7, s_state#9] Arguments: [s_store_sk#7, s_state#14], [s_store_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#14] -(16) CometBroadcastExchange +(18) CometBroadcastExchange Input [2]: [s_store_sk#7, s_state#14] Arguments: [s_store_sk#7, s_state#14] -(17) CometBroadcastHashJoin +(19) CometBroadcastHashJoin Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Right output [2]: [s_store_sk#7, s_state#14] Arguments: [ss_store_sk#10], [s_store_sk#7], Inner, BuildRight -(18) CometProject +(20) CometProject Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#7, s_state#14] Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14], [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] -(19) ReusedExchange [Reuses operator id: 6] +(21) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#5] -(20) CometBroadcastHashJoin +(22) CometBroadcastHashJoin Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] Right output [1]: [d_date_sk#5] Arguments: [ss_sold_date_sk#12], [d_date_sk#5], Inner, BuildRight -(21) CometProject +(23) CometProject Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14, d_date_sk#5] Arguments: [ss_net_profit#11, s_state#14], [ss_net_profit#11, s_state#14] -(22) CometHashAggregate +(24) CometHashAggregate Input [2]: [ss_net_profit#11, s_state#14] Keys [1]: [s_state#14] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] -(23) CometExchange +(25) CometExchange Input [2]: [s_state#14, sum#15] Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(24) CometHashAggregate +(26) CometHashAggregate Input [2]: [s_state#14, sum#15] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -(25) CometSort +(27) CometSort Input [3]: [s_state#14, _w0#16, s_state#14] Arguments: [s_state#14, _w0#16, s_state#14], [s_state#14 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] -(26) CometWindowExec +(28) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#14, _w0#16, s_state#14] -Arguments: [s_state#14, _w0#16, s_state#14, ranking#17], [rank(_w0#16) windowspecdefinition(s_state#14, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#14], [_w0#16 DESC NULLS LAST] -(27) CometFilter +(29) Window +Input [3]: [s_state#14, _w0#16, s_state#14] +Arguments: [rank(_w0#16) windowspecdefinition(s_state#14, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#14], [_w0#16 DESC NULLS LAST] + +(30) Filter [codegen id : 2] Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] Condition : (ranking#17 <= 5) -(28) CometProject +(31) Project [codegen id : 2] +Output [1]: [s_state#14] Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] -Arguments: [s_state#14], [s_state#14] -(29) CometBroadcastExchange +(32) BroadcastExchange Input [1]: [s_state#14] -Arguments: [s_state#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(30) CometBroadcastHashJoin -Left output [3]: [s_store_sk#7, s_county#8, s_state#9] -Right output [1]: [s_state#14] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)], [s_state#14], LeftSemi, BuildRight +(33) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] +Right keys [1]: [s_state#14] +Join type: LeftSemi +Join condition: None -(31) CometProject +(34) Project [codegen id : 3] +Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#18] Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Arguments: [s_store_sk#7, s_county#8, s_state#18], [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#18] -(32) CometBroadcastExchange +(35) BroadcastExchange Input [3]: [s_store_sk#7, s_county#8, s_state#18] -Arguments: [s_store_sk#7, s_county#8, s_state#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) CometBroadcastHashJoin -Left output [2]: [ss_store_sk#1, ss_net_profit#2] -Right output [3]: [s_store_sk#7, s_county#8, s_state#18] -Arguments: [ss_store_sk#1], [s_store_sk#7], Inner, BuildRight +(36) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#7] +Join type: Inner +Join condition: None -(34) CometProject +(37) Project [codegen id : 4] +Output [3]: [ss_net_profit#2, s_state#18, s_county#8] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#18] -Arguments: [ss_net_profit#2, s_state#18, s_county#8], [ss_net_profit#2, s_state#18, s_county#8] -(35) CometExpand +(38) Expand [codegen id : 4] Input [3]: [ss_net_profit#2, s_state#18, s_county#8] Arguments: [[ss_net_profit#2, s_state#18, s_county#8, 0], [ss_net_profit#2, s_state#18, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] -(36) CometHashAggregate +(39) HashAggregate [codegen id : 4] Input [4]: [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum#22] +Results [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] + +(40) CometColumnarExchange +Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] +Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(37) CometExchange -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#22] -Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(41) CometColumnarToRow [codegen id : 5] +Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] -(38) CometHashAggregate -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#22] +(42) HashAggregate [codegen id : 5] +Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS total_sum#25, s_state#19, s_county#20, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS lochierarchy#26, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS _w0#27, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS _w1#28, CASE WHEN (cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) = 0) THEN s_state#19 END AS _w2#29] -(39) CometExchange -Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: hashpartitioning(_w1#26, _w2#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(43) CometColumnarExchange +Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] +Arguments: hashpartitioning(_w1#28, _w2#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometSort -Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27], [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 DESC NULLS LAST] +(44) CometSort +Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] +Arguments: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29], [_w1#28 ASC NULLS FIRST, _w2#29 ASC NULLS FIRST, _w0#27 DESC NULLS LAST] -(41) CometWindowExec -Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28], [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 DESC NULLS LAST] +(45) CometColumnarToRow [codegen id : 6] +Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -(42) CometProject -Input [8]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] -Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28], [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] +(46) Window +Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] +Arguments: [rank(_w0#27) windowspecdefinition(_w1#28, _w2#29, _w0#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#30], [_w1#28, _w2#29], [_w0#27 DESC NULLS LAST] -(43) CometTakeOrderedAndProject -Input [5]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#24 DESC NULLS LAST,CASE WHEN (lochierarchy#24 = 0) THEN s_state#19 END ASC NULLS FIRST,rank_within_parent#28 ASC NULLS FIRST], output=[total_sum#23,s_state#19,s_county#20,lochierarchy#24,rank_within_parent#28]), [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28], 100, 0, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN s_state#19 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] +(47) Project [codegen id : 7] +Output [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] +Input [8]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29, rank_within_parent#30] -(44) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] +(48) TakeOrderedAndProject +Input [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] +Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#19 END ASC NULLS FIRST, rank_within_parent#30 ASC NULLS FIRST], [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (53) ++- * CometColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(50) CometFilter Input [2]: [d_date_sk#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(47) CometProject +(51) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) CometColumnarToRow [codegen id : 1] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(49) BroadcastExchange +(53) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt index e0eb970463..a67bf27b57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/extended.txt @@ -1,56 +1,60 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometHashAggregate - +- CometExchange + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt index b16765535b..d6ba61a844 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70.native_iceberg_compat/simplified.txt @@ -1,55 +1,73 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),spark_grouping_id] - CometExchange [s_state,s_county,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit] [s_state,s_county,spark_grouping_id,sum] - CometExpand [s_state,s_county] [ss_net_profit,s_state,s_county,spark_grouping_id] - CometProject [ss_net_profit,s_state,s_county] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_county,s_state] - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [s_store_sk,s_county,s_state] #5 - CometProject [s_state] [s_store_sk,s_county,s_state] - CometBroadcastHashJoin [s_store_sk,s_county,s_state,s_state] - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - CometBroadcastExchange [s_state] #6 - CometProject [s_state] - CometFilter [s_state,_w0,ranking] - CometWindowExec [s_state,_w0,ranking] - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #4 +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (7) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] + CometExchange [s_state] #7 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt index d059dabb26..d3610af0f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/explain.txt @@ -1,48 +1,52 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometProject (42) - +- CometWindowExec (41) - +- CometSort (40) - +- CometExchange (39) - +- CometHashAggregate (38) - +- CometExchange (37) - +- CometHashAggregate (36) - +- CometExpand (35) - +- CometProject (34) - +- CometBroadcastHashJoin (33) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - +- CometBroadcastExchange (32) - +- CometProject (31) - +- CometBroadcastHashJoin (30) - :- CometFilter (10) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - +- CometBroadcastExchange (29) - +- CometProject (28) - +- CometFilter (27) - +- CometWindowExec (26) - +- CometSort (25) - +- CometHashAggregate (24) - +- CometExchange (23) - +- CometHashAggregate (22) - +- CometProject (21) - +- CometBroadcastHashJoin (20) - :- CometProject (18) - : +- CometBroadcastHashJoin (17) - : :- CometFilter (12) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) - : +- CometBroadcastExchange (16) - : +- CometProject (15) - : +- CometFilter (14) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - +- ReusedExchange (19) +TakeOrderedAndProject (48) ++- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) + +- CometSort (44) + +- CometColumnarExchange (43) + +- * HashAggregate (42) + +- * CometColumnarToRow (41) + +- CometColumnarExchange (40) + +- * HashAggregate (39) + +- * Expand (38) + +- * Project (37) + +- * BroadcastHashJoin Inner BuildRight (36) + :- * CometColumnarToRow (9) + : +- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + +- BroadcastExchange (35) + +- * Project (34) + +- * BroadcastHashJoin LeftSemi BuildRight (33) + :- * CometColumnarToRow (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) + +- BroadcastExchange (32) + +- * Project (31) + +- * Filter (30) + +- Window (29) + +- * CometColumnarToRow (28) + +- CometSort (27) + +- CometHashAggregate (26) + +- CometExchange (25) + +- CometHashAggregate (24) + +- CometProject (23) + +- CometBroadcastHashJoin (22) + :- CometProject (20) + : +- CometBroadcastHashJoin (19) + : :- CometFilter (14) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) + : +- CometBroadcastExchange (18) + : +- CometProject (17) + : +- CometFilter (16) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) + +- ReusedExchange (21) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -85,18 +89,24 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(9) CometColumnarToRow [codegen id : 4] +Input [2]: [ss_store_sk#1, ss_net_profit#2] + +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(10) CometFilter +(11) CometFilter Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(12) CometColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] @@ -104,179 +114,191 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(12) CometFilter +(14) CometFilter Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Condition : isnotnull(ss_store_sk#10) -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [2]: [s_store_sk#7, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter +(16) CometFilter Input [2]: [s_store_sk#7, s_state#9] Condition : isnotnull(s_store_sk#7) -(15) CometProject +(17) CometProject Input [2]: [s_store_sk#7, s_state#9] Arguments: [s_store_sk#7, s_state#14], [s_store_sk#7, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#14] -(16) CometBroadcastExchange +(18) CometBroadcastExchange Input [2]: [s_store_sk#7, s_state#14] Arguments: [s_store_sk#7, s_state#14] -(17) CometBroadcastHashJoin +(19) CometBroadcastHashJoin Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Right output [2]: [s_store_sk#7, s_state#14] Arguments: [ss_store_sk#10], [s_store_sk#7], Inner, BuildRight -(18) CometProject +(20) CometProject Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#7, s_state#14] Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14], [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] -(19) ReusedExchange [Reuses operator id: 6] +(21) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#5] -(20) CometBroadcastHashJoin +(22) CometBroadcastHashJoin Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14] Right output [1]: [d_date_sk#5] Arguments: [ss_sold_date_sk#12], [d_date_sk#5], Inner, BuildRight -(21) CometProject +(23) CometProject Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#14, d_date_sk#5] Arguments: [ss_net_profit#11, s_state#14], [ss_net_profit#11, s_state#14] -(22) CometHashAggregate +(24) CometHashAggregate Input [2]: [ss_net_profit#11, s_state#14] Keys [1]: [s_state#14] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] -(23) CometExchange +(25) CometExchange Input [2]: [s_state#14, sum#15] Arguments: hashpartitioning(s_state#14, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(24) CometHashAggregate +(26) CometHashAggregate Input [2]: [s_state#14, sum#15] Keys [1]: [s_state#14] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -(25) CometSort +(27) CometSort Input [3]: [s_state#14, _w0#16, s_state#14] Arguments: [s_state#14, _w0#16, s_state#14], [s_state#14 ASC NULLS FIRST, _w0#16 DESC NULLS LAST] -(26) CometWindowExec +(28) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#14, _w0#16, s_state#14] -Arguments: [s_state#14, _w0#16, s_state#14, ranking#17], [rank(_w0#16) windowspecdefinition(s_state#14, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#14], [_w0#16 DESC NULLS LAST] -(27) CometFilter +(29) Window +Input [3]: [s_state#14, _w0#16, s_state#14] +Arguments: [rank(_w0#16) windowspecdefinition(s_state#14, _w0#16 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#17], [s_state#14], [_w0#16 DESC NULLS LAST] + +(30) Filter [codegen id : 2] Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] Condition : (ranking#17 <= 5) -(28) CometProject +(31) Project [codegen id : 2] +Output [1]: [s_state#14] Input [4]: [s_state#14, _w0#16, s_state#14, ranking#17] -Arguments: [s_state#14], [s_state#14] -(29) CometBroadcastExchange +(32) BroadcastExchange Input [1]: [s_state#14] -Arguments: [s_state#14] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(30) CometBroadcastHashJoin -Left output [3]: [s_store_sk#7, s_county#8, s_state#9] -Right output [1]: [s_state#14] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)], [s_state#14], LeftSemi, BuildRight +(33) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] +Right keys [1]: [s_state#14] +Join type: LeftSemi +Join condition: None -(31) CometProject +(34) Project [codegen id : 3] +Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#18] Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Arguments: [s_store_sk#7, s_county#8, s_state#18], [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#18] -(32) CometBroadcastExchange +(35) BroadcastExchange Input [3]: [s_store_sk#7, s_county#8, s_state#18] -Arguments: [s_store_sk#7, s_county#8, s_state#18] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) CometBroadcastHashJoin -Left output [2]: [ss_store_sk#1, ss_net_profit#2] -Right output [3]: [s_store_sk#7, s_county#8, s_state#18] -Arguments: [ss_store_sk#1], [s_store_sk#7], Inner, BuildRight +(36) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#7] +Join type: Inner +Join condition: None -(34) CometProject +(37) Project [codegen id : 4] +Output [3]: [ss_net_profit#2, s_state#18, s_county#8] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#18] -Arguments: [ss_net_profit#2, s_state#18, s_county#8], [ss_net_profit#2, s_state#18, s_county#8] -(35) CometExpand +(38) Expand [codegen id : 4] Input [3]: [ss_net_profit#2, s_state#18, s_county#8] Arguments: [[ss_net_profit#2, s_state#18, s_county#8, 0], [ss_net_profit#2, s_state#18, null, 1], [ss_net_profit#2, null, null, 3]], [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] -(36) CometHashAggregate +(39) HashAggregate [codegen id : 4] Input [4]: [ss_net_profit#2, s_state#19, s_county#20, spark_grouping_id#21] Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum#22] +Results [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] + +(40) CometColumnarExchange +Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] +Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(37) CometExchange -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#22] -Arguments: hashpartitioning(s_state#19, s_county#20, spark_grouping_id#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(41) CometColumnarToRow [codegen id : 5] +Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] -(38) CometHashAggregate -Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#22] +(42) HashAggregate [codegen id : 5] +Input [4]: [s_state#19, s_county#20, spark_grouping_id#21, sum#23] Keys [3]: [s_state#19, s_county#20, spark_grouping_id#21] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] +Results [7]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS total_sum#25, s_state#19, s_county#20, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS lochierarchy#26, MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) AS _w0#27, (cast((shiftright(spark_grouping_id#21, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint)) AS _w1#28, CASE WHEN (cast((shiftright(spark_grouping_id#21, 0) & 1) as tinyint) = 0) THEN s_state#19 END AS _w2#29] -(39) CometExchange -Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: hashpartitioning(_w1#26, _w2#27, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(43) CometColumnarExchange +Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] +Arguments: hashpartitioning(_w1#28, _w2#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(40) CometSort -Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27], [_w1#26 ASC NULLS FIRST, _w2#27 ASC NULLS FIRST, _w0#25 DESC NULLS LAST] +(44) CometSort +Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] +Arguments: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29], [_w1#28 ASC NULLS FIRST, _w2#29 ASC NULLS FIRST, _w0#27 DESC NULLS LAST] -(41) CometWindowExec -Input [7]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27] -Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28], [rank(_w0#25) windowspecdefinition(_w1#26, _w2#27, _w0#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#28], [_w1#26, _w2#27], [_w0#25 DESC NULLS LAST] +(45) CometColumnarToRow [codegen id : 6] +Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] -(42) CometProject -Input [8]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, _w0#25, _w1#26, _w2#27, rank_within_parent#28] -Arguments: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28], [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] +(46) Window +Input [7]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29] +Arguments: [rank(_w0#27) windowspecdefinition(_w1#28, _w2#29, _w0#27 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#30], [_w1#28, _w2#29], [_w0#27 DESC NULLS LAST] -(43) CometTakeOrderedAndProject -Input [5]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#24 DESC NULLS LAST,CASE WHEN (lochierarchy#24 = 0) THEN s_state#19 END ASC NULLS FIRST,rank_within_parent#28 ASC NULLS FIRST], output=[total_sum#23,s_state#19,s_county#20,lochierarchy#24,rank_within_parent#28]), [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28], 100, 0, [lochierarchy#24 DESC NULLS LAST, CASE WHEN (lochierarchy#24 = 0) THEN s_state#19 END ASC NULLS FIRST, rank_within_parent#28 ASC NULLS FIRST], [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] +(47) Project [codegen id : 7] +Output [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] +Input [8]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, _w0#27, _w1#28, _w2#29, rank_within_parent#30] -(44) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#23, s_state#19, s_county#20, lochierarchy#24, rank_within_parent#28] +(48) TakeOrderedAndProject +Input [5]: [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] +Arguments: 100, [lochierarchy#26 DESC NULLS LAST, CASE WHEN (lochierarchy#26 = 0) THEN s_state#19 END ASC NULLS FIRST, rank_within_parent#30 ASC NULLS FIRST], [total_sum#25, s_state#19, s_county#20, lochierarchy#26, rank_within_parent#30] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (49) -+- * CometColumnarToRow (48) - +- CometProject (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (53) ++- * CometColumnarToRow (52) + +- CometProject (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1200), LessThanOrEqual(d_month_seq,1211), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(50) CometFilter Input [2]: [d_date_sk#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1200)) AND (d_month_seq#6 <= 1211)) AND isnotnull(d_date_sk#5)) -(47) CometProject +(51) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(48) CometColumnarToRow [codegen id : 1] +(52) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(49) BroadcastExchange +(53) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6] -Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt index e0eb970463..a67bf27b57 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/extended.txt @@ -1,56 +1,60 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometExpand - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometHashAggregate - +- CometExchange + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Expand + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 49 out of 52 eligible operators (94%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 34 out of 52 eligible operators (65%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt index b16765535b..d6ba61a844 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q70/simplified.txt @@ -1,55 +1,73 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ss_net_profit)),spark_grouping_id] - CometExchange [s_state,s_county,spark_grouping_id] #2 - CometHashAggregate [ss_net_profit] [s_state,s_county,spark_grouping_id,sum] - CometExpand [s_state,s_county] [ss_net_profit,s_state,s_county,spark_grouping_id] - CometProject [ss_net_profit,s_state,s_county] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_county,s_state] - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [s_store_sk,s_county,s_state] #5 - CometProject [s_state] [s_store_sk,s_county,s_state] - CometBroadcastHashJoin [s_store_sk,s_county,s_state,s_state] - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - CometBroadcastExchange [s_state] #6 - CometProject [s_state] - CometFilter [s_state,_w0,ranking] - CometWindowExec [s_state,_w0,ranking] - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #7 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #8 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #4 +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (7) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometSort [total_sum,s_state,s_county,lochierarchy,_w0,_w1,_w2] + CometColumnarExchange [_w1,_w2] #1 + WholeStageCodegen (5) + HashAggregate [s_state,s_county,spark_grouping_id,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,lochierarchy,_w0,_w1,_w2,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county,spark_grouping_id] #2 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,spark_grouping_id,ss_net_profit] [sum,sum] + Expand [ss_net_profit,s_state,s_county] + Project [ss_net_profit,s_state,s_county] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #5 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] + CometExchange [s_state] #7 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #8 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt index 20b60287be..1af5e2612c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometProject (22) - +- CometWindowExec (21) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) +- CometSort (20) +- CometExchange (19) +- CometHashAggregate (18) @@ -119,20 +119,20 @@ Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNative Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 1] Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21], [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] -(22) CometProject -Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] -Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +(22) Window +Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] -(23) CometTakeOrderedAndProject -Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#17 DESC NULLS LAST,CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST,rank_within_parent#21 ASC NULLS FIRST], output=[total_sum#16,i_category#12,i_class#13,lochierarchy#17,rank_within_parent#21]), [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21], 100, 0, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +(23) Project [codegen id : 2] +Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] -(24) CometColumnarToRow [codegen id : 1] +(24) TakeOrderedAndProject Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt index f919130b53..0b05b63e05 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt index a6b54bd89f..29e2d72920 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86.native_iceberg_compat/simplified.txt @@ -1,34 +1,36 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] - CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (2) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt index 20b60287be..1af5e2612c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometProject (22) - +- CometWindowExec (21) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) +- CometSort (20) +- CometExchange (19) +- CometHashAggregate (18) @@ -119,20 +119,20 @@ Arguments: hashpartitioning(_w1#19, _w2#20, 5), ENSURE_REQUIREMENTS, CometNative Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20], [_w1#19 ASC NULLS FIRST, _w2#20 ASC NULLS FIRST, _w0#18 DESC NULLS LAST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 1] Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] -Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21], [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] -(22) CometProject -Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] -Arguments: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +(22) Window +Input [7]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20] +Arguments: [rank(_w0#18) windowspecdefinition(_w1#19, _w2#20, _w0#18 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#21], [_w1#19, _w2#20], [_w0#18 DESC NULLS LAST] -(23) CometTakeOrderedAndProject -Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#17 DESC NULLS LAST,CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST,rank_within_parent#21 ASC NULLS FIRST], output=[total_sum#16,i_category#12,i_class#13,lochierarchy#17,rank_within_parent#21]), [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21], 100, 0, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +(23) Project [codegen id : 2] +Output [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +Input [8]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, _w0#18, _w1#19, _w2#20, rank_within_parent#21] -(24) CometColumnarToRow [codegen id : 1] +(24) TakeOrderedAndProject Input [5]: [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] +Arguments: 100, [lochierarchy#17 DESC NULLS LAST, CASE WHEN (lochierarchy#17 = 0) THEN i_category#12 END ASC NULLS FIRST, rank_within_parent#21 ASC NULLS FIRST], [total_sum#16, i_category#12, i_class#13, lochierarchy#17, rank_within_parent#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt index f919130b53..0b05b63e05 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt index a6b54bd89f..29e2d72920 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q86/simplified.txt @@ -1,34 +1,36 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,rank_within_parent] - CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] - CometExchange [_w1,_w2] #1 - CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] - CometExchange [i_category,i_class,spark_grouping_id] #2 - CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] - CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] - CometProject [ws_net_paid,i_category,i_class] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #4 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #5 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (2) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [_w0,_w1,_w2] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2] + CometExchange [_w1,_w2] #1 + CometHashAggregate [sum] [total_sum,i_category,i_class,lochierarchy,_w0,_w1,_w2,sum(UnscaledValue(ws_net_paid)),spark_grouping_id] + CometExchange [i_category,i_class,spark_grouping_id] #2 + CometHashAggregate [ws_net_paid] [i_category,i_class,spark_grouping_id,sum] + CometExpand [i_category,i_class] [ws_net_paid,i_category,i_class,spark_grouping_id] + CometProject [ws_net_paid,i_category,i_class] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #4 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #5 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt index 1de0dfa34d..47ce0d4a91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometProject (27) - +- CometFilter (26) - +- CometWindowExec (25) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_ Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22], [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] -(26) CometFilter +(26) Window +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] + +(27) Filter [codegen id : 2] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END -(27) CometProject +(28) Project [codegen id : 2] +Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] - -(28) CometTakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST,s_store_name#17 ASC NULLS FIRST], output=[i_category#7,i_class#6,i_brand#5,s_store_name#17,s_company_name#18,d_moy#15,sum_sales#20,avg_monthly_sales#22]), [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22], 100, 0, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -(29) CometColumnarToRow [codegen id : 1] +(29) TakeOrderedAndProject Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] +Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt index 8adffb63e3..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt index 682d63a75b..61bfd1d960 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.native_iceberg_compat/simplified.txt @@ -1,39 +1,41 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - CometProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - CometFilter [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] - CometWindowExec [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] + WholeStageCodegen (2) + Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt index 1de0dfa34d..47ce0d4a91 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt @@ -1,9 +1,9 @@ == Physical Plan == -* CometColumnarToRow (29) -+- CometTakeOrderedAndProject (28) - +- CometProject (27) - +- CometFilter (26) - +- CometWindowExec (25) +TakeOrderedAndProject (29) ++- * Project (28) + +- * Filter (27) + +- Window (26) + +- * CometColumnarToRow (25) +- CometSort (24) +- CometExchange (23) +- CometHashAggregate (22) @@ -144,24 +144,24 @@ Arguments: hashpartitioning(i_category#7, i_brand#5, s_store_name#17, s_company_ Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21], [i_category#7 ASC NULLS FIRST, i_brand#5 ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST, s_company_name#18 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22], [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] -(26) CometFilter +(26) Window +Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21] +Arguments: [avg(_w0#21) windowspecdefinition(i_category#7, i_brand#5, s_store_name#17, s_company_name#18, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#7, i_brand#5, s_store_name#17, s_company_name#18] + +(27) Filter [codegen id : 2] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] Condition : CASE WHEN NOT (avg_monthly_sales#22 = 0.000000) THEN ((abs((sum_sales#20 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END -(27) CometProject +(28) Project [codegen id : 2] +Output [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] Input [9]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, _w0#21, avg_monthly_sales#22] -Arguments: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] - -(28) CometTakeOrderedAndProject -Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST,s_store_name#17 ASC NULLS FIRST], output=[i_category#7,i_class#6,i_brand#5,s_store_name#17,s_company_name#18,d_moy#15,sum_sales#20,avg_monthly_sales#22]), [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22], 100, 0, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] -(29) CometColumnarToRow [codegen id : 1] +(29) TakeOrderedAndProject Input [8]: [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] +Arguments: 100, [(sum_sales#20 - avg_monthly_sales#22) ASC NULLS FIRST, s_store_name#17 ASC NULLS FIRST], [i_category#7, i_class#6, i_brand#5, s_store_name#17, s_company_name#18, d_moy#15, sum_sales#20, avg_monthly_sales#22] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt index 8adffb63e3..004828cdfb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/extended.txt @@ -1,8 +1,8 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -34,4 +34,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 31 out of 33 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 27 out of 33 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt index 682d63a75b..61bfd1d960 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/simplified.txt @@ -1,39 +1,41 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - CometProject [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] - CometFilter [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] - CometWindowExec [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,avg_monthly_sales] - CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] - CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] - CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] - CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] - CometFilter [i_item_sk,i_brand,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_moy] #5 - CometProject [d_date_sk,d_moy] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,s_store_name,i_category,i_class,i_brand,s_company_name,d_moy] + WholeStageCodegen (2) + Project [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,avg_monthly_sales] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_class,i_brand,s_store_name,s_company_name,d_moy,sum] + CometProject [i_brand,i_class,i_category,ss_sales_price,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,d_moy] + CometBroadcastHashJoin [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_moy] + CometProject [i_brand,i_class,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_class,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_class,i_category] [i_item_sk,i_brand,i_class,i_category] + CometFilter [i_item_sk,i_brand,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_moy] #5 + CometProject [d_date_sk,d_moy] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt index 20a3805afe..4cc725ef2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/explain.txt @@ -1,29 +1,30 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometProject (24) - +- CometSort (23) - +- CometExchange (22) - +- CometProject (21) - +- CometWindowExec (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (26) ++- CometProject (25) + +- CometSort (24) + +- CometColumnarExchange (23) + +- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -116,58 +117,61 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] + +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(22) CometExchange +(23) CometColumnarExchange Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometSort +(24) CometSort Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] -(24) CometProject +(25) CometProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -(25) CometColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 3] Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +BroadcastExchange (31) ++- * CometColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(28) CometProject +(29) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt index c84faeb01d..c34e3cfede 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/extended.txt @@ -1,33 +1,34 @@ CometColumnarToRow +- CometProject +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt index 6f23d4be47..af05a7d8c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.native_iceberg_compat/simplified.txt @@ -1,35 +1,40 @@ -WholeStageCodegen (1) +WholeStageCodegen (3) CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (2) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt index 20a3805afe..4cc725ef2b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt @@ -1,29 +1,30 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometProject (24) - +- CometSort (23) - +- CometExchange (22) - +- CometProject (21) - +- CometWindowExec (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (26) ++- CometProject (25) + +- CometSort (24) + +- CometColumnarExchange (23) + +- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -116,58 +117,61 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject +(21) Window +Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] + +(22) Project [codegen id : 2] +Output [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] Input [8]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, i_item_id#11, _we0#19] -Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20, i_item_id#11] -(22) CometExchange +(23) CometColumnarExchange Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometSort +(24) CometSort Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] -(24) CometProject +(25) CometProject Input [7]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20, i_item_id#11] Arguments: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -(25) CometColumnarToRow [codegen id : 1] +(26) CometColumnarToRow [codegen id : 3] Input [6]: [i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +BroadcastExchange (31) ++- * CometColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(28) CometProject +(29) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt index c84faeb01d..c34e3cfede 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/extended.txt @@ -1,33 +1,34 @@ CometColumnarToRow +- CometProject +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 27 out of 29 eligible operators (93%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 25 out of 29 eligible operators (86%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt index 6f23d4be47..af05a7d8c6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/simplified.txt @@ -1,35 +1,40 @@ -WholeStageCodegen (1) +WholeStageCodegen (3) CometColumnarToRow InputAdapter CometProject [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - CometProject [_w0,_we0] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio,i_item_id] - CometWindowExec [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,_we0] - CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (2) + Project [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0,i_item_id] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id] + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,i_item_id,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt index 2708ea9eae..9ddd3aeb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt index 2708ea9eae..9ddd3aeb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q12/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt index b08f4f6d0e..400633ed9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt index b08f4f6d0e..400633ed9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q20/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt index f0fa5a92af..dee6669b47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt index f0fa5a92af..dee6669b47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q36a/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt index 1bbdb7273f..11fbf5bd11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47.native_iceberg_compat/extended.txt @@ -7,7 +7,7 @@ : : +- Filter : : +- Window : : +- Filter - : : +- Window + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -39,7 +39,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange : +- Project - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -71,7 +71,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt index 1bbdb7273f..11fbf5bd11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q47/extended.txt @@ -7,7 +7,7 @@ : : +- Filter : : +- Window : : +- Filter - : : +- Window + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -39,7 +39,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store : +- BroadcastExchange : +- Project - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -71,7 +71,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store +- BroadcastExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt index 4988a26564..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49.native_iceberg_compat/extended.txt @@ -8,7 +8,7 @@ CometColumnarToRow : +- Filter : +- Window : +- Sort - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -40,7 +40,7 @@ CometColumnarToRow : +- Filter : +- Window : +- Sort - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -67,7 +67,7 @@ CometColumnarToRow +- Filter +- Window +- Sort - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt index 4988a26564..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q49/extended.txt @@ -8,7 +8,7 @@ CometColumnarToRow : +- Filter : +- Window : +- Sort - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -40,7 +40,7 @@ CometColumnarToRow : +- Filter : +- Window : +- Sort - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -67,7 +67,7 @@ CometColumnarToRow +- Filter +- Window +- Sort - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt index 86caa88151..15a87f66b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a.native_iceberg_compat/extended.txt @@ -4,7 +4,7 @@ +- HashAggregate +- Project +- BroadcastHashJoin - :- Window [COMET: Partitioning and sorting specifications must be the same.] + :- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -20,7 +20,7 @@ : : +- Project : : +- BroadcastHashJoin : : :- Project - : : : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : : : +- Window [COMET: Window expressions are not supported] : : : +- CometColumnarToRow : : : +- CometSort : : : +- CometExchange @@ -43,7 +43,7 @@ : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -73,7 +73,7 @@ : +- Project : +- BroadcastHashJoin : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -91,7 +91,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -109,7 +109,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange @@ -125,7 +125,7 @@ : +- Project : +- BroadcastHashJoin : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -148,7 +148,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -178,7 +178,7 @@ +- Project +- BroadcastHashJoin :- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -196,7 +196,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt index 86caa88151..15a87f66b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q51a/extended.txt @@ -4,7 +4,7 @@ +- HashAggregate +- Project +- BroadcastHashJoin - :- Window [COMET: Partitioning and sorting specifications must be the same.] + :- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -20,7 +20,7 @@ : : +- Project : : +- BroadcastHashJoin : : :- Project - : : : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : : : +- Window [COMET: Window expressions are not supported] : : : +- CometColumnarToRow : : : +- CometSort : : : +- CometExchange @@ -43,7 +43,7 @@ : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : : +- BroadcastExchange : : +- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -73,7 +73,7 @@ : +- Project : +- BroadcastHashJoin : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -91,7 +91,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -109,7 +109,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange @@ -125,7 +125,7 @@ : +- Project : +- BroadcastHashJoin : :- Project - : : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -148,7 +148,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim : +- BroadcastExchange : +- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -178,7 +178,7 @@ +- Project +- BroadcastHashJoin :- Project - : +- Window [COMET: Partitioning and sorting specifications must be the same.] + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -196,7 +196,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- BroadcastExchange +- Project - +- Window [COMET: Partitioning and sorting specifications must be the same.] + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt index 101b15d3ac..7bee0e61e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57.native_iceberg_compat/extended.txt @@ -7,7 +7,7 @@ : : +- Filter : : +- Window : : +- Filter - : : +- Window + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -39,7 +39,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -71,7 +71,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center +- BroadcastExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt index 101b15d3ac..7bee0e61e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q57/extended.txt @@ -7,7 +7,7 @@ : : +- Filter : : +- Window : : +- Filter - : : +- Window + : : +- Window [COMET: Window expressions are not supported] : : +- CometColumnarToRow : : +- CometSort : : +- CometExchange @@ -39,7 +39,7 @@ : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center : +- BroadcastExchange : +- Project - : +- Window + : +- Window [COMET: Window expressions are not supported] : +- CometColumnarToRow : +- CometSort : +- CometExchange @@ -71,7 +71,7 @@ : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center +- BroadcastExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt index 7192d90eb0..1574f09c56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt index 7192d90eb0..1574f09c56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q70a/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt index 6f5a5be275..6dd65034ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a.native_iceberg_compat/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt index 6f5a5be275..6dd65034ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q86a/extended.txt @@ -1,6 +1,6 @@ TakeOrderedAndProject [COMET: ] +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt index 30105bb26b..54a4d3c1b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98.native_iceberg_compat/extended.txt @@ -2,7 +2,7 @@ CometColumnarToRow +- CometSort +- CometColumnarExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt index 30105bb26b..54a4d3c1b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q98/extended.txt @@ -2,7 +2,7 @@ CometColumnarToRow +- CometSort +- CometColumnarExchange +- Project - +- Window + +- Window [COMET: Window expressions are not supported] +- CometColumnarToRow +- CometSort +- CometExchange diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt index 55482c8d8d..5737af1936 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometProject (22) - +- CometWindowExec (21) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +(22) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(23) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) Project [codegen id : 3] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] -(24) CometColumnarToRow [codegen id : 2] +(24) TakeOrderedAndProject Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt index 8b95fbaf2c..6a604adb85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS _w0#9)] @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt index 8a81ec996c..95364332b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12.native_iceberg_compat/simplified.txt @@ -1,36 +1,38 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (3) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt index 55482c8d8d..5737af1936 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometProject (22) - +- CometWindowExec (21) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +(22) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(23) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) Project [codegen id : 3] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] -(24) CometColumnarToRow [codegen id : 2] +(24) TakeOrderedAndProject Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt index 8b95fbaf2c..6a604adb85 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#6))#7,17,2) AS _w0#9)] @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt index 8a81ec996c..95364332b9 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q12/simplified.txt @@ -1,36 +1,38 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (3) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ws_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt index a11e8312cf..78aa0a8ea8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometProject (22) - +- CometWindowExec (21) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +(22) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(23) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) Project [codegen id : 3] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] -(24) CometColumnarToRow [codegen id : 2] +(24) TakeOrderedAndProject Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt index 7f6cd6b5e3..956b109103 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS _w0#9)] @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt index 390dd3ad8c..1b6a5f13f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20.native_iceberg_compat/simplified.txt @@ -1,36 +1,38 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (3) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt index a11e8312cf..78aa0a8ea8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometTakeOrderedAndProject (23) - +- CometProject (22) - +- CometWindowExec (21) +TakeOrderedAndProject (24) ++- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) +- CometSort (20) +- CometColumnarExchange (19) +- * HashAggregate (18) @@ -120,20 +120,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +(22) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(23) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#21 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#18,revenueratio#21]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +(23) Project [codegen id : 3] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] -(24) CometColumnarToRow [codegen id : 2] +(24) TakeOrderedAndProject Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt index 7f6cd6b5e3..956b109103 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#6))#7,17,2) AS _w0#9)] @@ -29,4 +29,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 24 out of 27 eligible operators (88%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 21 out of 27 eligible operators (77%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt index 390dd3ad8c..1b6a5f13f3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q20/simplified.txt @@ -1,36 +1,38 @@ -WholeStageCodegen (2) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #1 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (3) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #1 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(cs_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt index 58eeeab0ef..b8a2a2bb8d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (48) -+- CometTakeOrderedAndProject (47) - +- CometProject (46) - +- CometWindowExec (45) +TakeOrderedAndProject (48) ++- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) +- CometSort (44) +- CometExchange (43) +- CometHashAggregate (42) @@ -258,20 +258,20 @@ Arguments: hashpartitioning(lochierarchy#25, _w0#75, 5), ENSURE_REQUIREMENTS, Co Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75], [lochierarchy#25 ASC NULLS FIRST, _w0#75 ASC NULLS FIRST, gross_margin#20 ASC NULLS FIRST] -(45) CometWindowExec +(45) CometColumnarToRow [codegen id : 7] Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] -Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76], [rank(gross_margin#20) windowspecdefinition(lochierarchy#25, _w0#75, gross_margin#20 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#76], [lochierarchy#25, _w0#75], [gross_margin#20 ASC NULLS FIRST] -(46) CometProject -Input [6]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76] -Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +(46) Window +Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] +Arguments: [rank(gross_margin#20) windowspecdefinition(lochierarchy#25, _w0#75, gross_margin#20 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#76], [lochierarchy#25, _w0#75], [gross_margin#20 ASC NULLS FIRST] -(47) CometTakeOrderedAndProject -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#25 DESC NULLS LAST,CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST,rank_within_parent#76 ASC NULLS FIRST], output=[gross_margin#20,i_category#21,i_class#22,lochierarchy#25,rank_within_parent#76]), [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76], 100, 0, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST, rank_within_parent#76 ASC NULLS FIRST], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +(47) Project [codegen id : 8] +Output [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +Input [6]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76] -(48) CometColumnarToRow [codegen id : 7] +(48) TakeOrderedAndProject Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST, rank_within_parent#76 ASC NULLS FIRST], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt index 197fa38310..39168c9476 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -107,4 +107,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 84 out of 99 eligible operators (84%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 99 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt index d78d4c6456..b799504154 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a.native_iceberg_compat/simplified.txt @@ -1,70 +1,72 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,rank_within_parent] - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (6) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (3) - HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #8 - WholeStageCodegen (2) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (5) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (4) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (8) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [gross_margin,lochierarchy,_w0] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + WholeStageCodegen (6) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (1) + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class] #3 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (3) + HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #8 + WholeStageCodegen (2) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 + WholeStageCodegen (5) + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #9 + WholeStageCodegen (4) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt index 58eeeab0ef..b8a2a2bb8d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (48) -+- CometTakeOrderedAndProject (47) - +- CometProject (46) - +- CometWindowExec (45) +TakeOrderedAndProject (48) ++- * Project (47) + +- Window (46) + +- * CometColumnarToRow (45) +- CometSort (44) +- CometExchange (43) +- CometHashAggregate (42) @@ -258,20 +258,20 @@ Arguments: hashpartitioning(lochierarchy#25, _w0#75, 5), ENSURE_REQUIREMENTS, Co Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75], [lochierarchy#25 ASC NULLS FIRST, _w0#75 ASC NULLS FIRST, gross_margin#20 ASC NULLS FIRST] -(45) CometWindowExec +(45) CometColumnarToRow [codegen id : 7] Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] -Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76], [rank(gross_margin#20) windowspecdefinition(lochierarchy#25, _w0#75, gross_margin#20 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#76], [lochierarchy#25, _w0#75], [gross_margin#20 ASC NULLS FIRST] -(46) CometProject -Input [6]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76] -Arguments: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +(46) Window +Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75] +Arguments: [rank(gross_margin#20) windowspecdefinition(lochierarchy#25, _w0#75, gross_margin#20 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#76], [lochierarchy#25, _w0#75], [gross_margin#20 ASC NULLS FIRST] -(47) CometTakeOrderedAndProject -Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#25 DESC NULLS LAST,CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST,rank_within_parent#76 ASC NULLS FIRST], output=[gross_margin#20,i_category#21,i_class#22,lochierarchy#25,rank_within_parent#76]), [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76], 100, 0, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST, rank_within_parent#76 ASC NULLS FIRST], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +(47) Project [codegen id : 8] +Output [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +Input [6]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, _w0#75, rank_within_parent#76] -(48) CometColumnarToRow [codegen id : 7] +(48) TakeOrderedAndProject Input [5]: [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] +Arguments: 100, [lochierarchy#25 DESC NULLS LAST, CASE WHEN (lochierarchy#25 = 0) THEN i_category#21 END ASC NULLS FIRST, rank_within_parent#76 ASC NULLS FIRST], [gross_margin#20, i_category#21, i_class#22, lochierarchy#25, rank_within_parent#76] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt index 197fa38310..39168c9476 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -107,4 +107,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 84 out of 99 eligible operators (84%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 81 out of 99 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt index d78d4c6456..b799504154 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q36a/simplified.txt @@ -1,70 +1,72 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,rank_within_parent] - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - WholeStageCodegen (6) - HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - WholeStageCodegen (3) - HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #8 - WholeStageCodegen (2) - HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 - WholeStageCodegen (5) - HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #9 - WholeStageCodegen (4) - HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum,sum] #3 +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (8) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [gross_margin,lochierarchy,_w0] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometColumnarExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + WholeStageCodegen (6) + HashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (1) + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class] #3 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + WholeStageCodegen (3) + HashAggregate [i_category,sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #8 + WholeStageCodegen (2) + HashAggregate [i_category,ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 + WholeStageCodegen (5) + HashAggregate [sum,isEmpty,sum,isEmpty] [sum(ss_net_profit),sum(ss_ext_sales_price),gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum,isEmpty,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #9 + WholeStageCodegen (4) + HashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum,sum] [sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),ss_net_profit,ss_ext_sales_price,sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt index 8b6d6fe277..b72fd62948 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/explain.txt @@ -1,48 +1,54 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (29) - : : +- CometFilter (28) - : : +- CometWindowExec (27) - : : +- CometFilter (26) - : : +- CometWindowExec (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometWindowExec (35) - : +- CometSort (34) - : +- CometColumnarExchange (33) - : +- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- ReusedExchange (30) - +- ReusedExchange (40) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -160,112 +166,136 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 2] Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometFilter +(26) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(27) Filter [codegen id : 3] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) CometWindowExec +(28) Window Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22], [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(28) CometFilter +(29) Filter [codegen id : 10] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(29) CometProject +(30) Project [codegen id : 10] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -(30) ReusedExchange [Reuses operator id: 20] +(31) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(31) CometColumnarToRow [codegen id : 2] +(32) CometColumnarToRow [codegen id : 4] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(32) HashAggregate [codegen id : 2] +(33) HashAggregate [codegen id : 4] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] -(33) CometColumnarExchange +(34) CometColumnarExchange Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(34) CometSort +(35) CometSort Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(35) CometWindowExec +(36) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] + +(37) Window Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32], [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(36) CometProject +(38) Project [codegen id : 6] +Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(37) CometBroadcastExchange +(39) BroadcastExchange Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(38) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Right output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)], Inner, BuildRight +(40) BroadcastHashJoin [codegen id : 10] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] +Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)] +Join type: Inner +Join condition: None -(39) CometProject +(41) Project [codegen id : 10] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -(40) ReusedExchange [Reuses operator id: 37] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +(42) ReusedExchange [Reuses operator id: 34] +Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] + +(43) CometSort +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] + +(44) CometColumnarToRow [codegen id : 8] +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] + +(45) Window +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] + +(46) Project [codegen id : 9] +Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] -(41) CometBroadcastHashJoin -Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -Right output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#38 - 1)], Inner, BuildRight +(47) BroadcastExchange +Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(42) CometProject -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -Arguments: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#37 AS nsum#40] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] +Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 1)] +Join type: Inner +Join condition: None -(43) CometTakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(49) Project [codegen id : 10] +Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -(44) CometColumnarToRow [codegen id : 4] -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(50) TakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(52) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(47) CometColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(48) BroadcastExchange +(54) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt index eeae7c1e95..1a4201332c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/extended.txt @@ -1,78 +1,80 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] @@ -103,4 +105,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt index 77ba154cab..b08d56b5ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47.native_iceberg_compat/simplified.txt @@ -1,57 +1,77 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 - WholeStageCodegen (2) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] + WholeStageCodegen (10) + Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (3) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (2) CometColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt index 8b6d6fe277..b72fd62948 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/explain.txt @@ -1,48 +1,54 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (29) - : : +- CometFilter (28) - : : +- CometWindowExec (27) - : : +- CometFilter (26) - : : +- CometWindowExec (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometWindowExec (35) - : +- CometSort (34) - : +- CometColumnarExchange (33) - : +- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- ReusedExchange (30) - +- ReusedExchange (40) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -160,112 +166,136 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 2] Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometFilter +(26) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(27) Filter [codegen id : 3] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) CometWindowExec +(28) Window Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22], [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [avg(_w0#20) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#22], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(28) CometFilter +(29) Filter [codegen id : 10] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] Condition : ((isnotnull(avg_monthly_sales#22) AND (avg_monthly_sales#22 > 0.000000)) AND CASE WHEN (avg_monthly_sales#22 > 0.000000) THEN ((abs((sum_sales#19 - avg_monthly_sales#22)) / avg_monthly_sales#22) > 0.1000000000000000) END) -(29) CometProject +(30) Project [codegen id : 10] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, _w0#20, rn#21, avg_monthly_sales#22] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -(30) ReusedExchange [Reuses operator id: 20] +(31) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(31) CometColumnarToRow [codegen id : 2] +(32) CometColumnarToRow [codegen id : 4] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] -(32) HashAggregate [codegen id : 2] +(33) HashAggregate [codegen id : 4] Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum#29] Keys [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28] Functions [1]: [sum(UnscaledValue(ss_sales_price#30))] Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#30))#18] Results [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, MakeDecimal(sum(UnscaledValue(ss_sales_price#30))#18,17,2) AS sum_sales#31] -(33) CometColumnarExchange +(34) CometColumnarExchange Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: hashpartitioning(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(34) CometSort +(35) CometSort Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31], [i_category#23 ASC NULLS FIRST, i_brand#24 ASC NULLS FIRST, s_store_name#25 ASC NULLS FIRST, s_company_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(35) CometWindowExec +(36) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] + +(37) Window Input [7]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32], [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] +Arguments: [rank(d_year#27, d_moy#28) windowspecdefinition(i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26], [d_year#27 ASC NULLS FIRST, d_moy#28 ASC NULLS FIRST] -(36) CometProject +(38) Project [codegen id : 6] +Output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] Input [8]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, d_year#27, d_moy#28, sum_sales#31, rn#32] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -(37) CometBroadcastExchange +(39) BroadcastExchange Input [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(38) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21] -Right output [6]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)], Inner, BuildRight +(40) BroadcastHashJoin [codegen id : 10] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] +Right keys [5]: [i_category#23, i_brand#24, s_store_name#25, s_company_name#26, (rn#32 + 1)] +Join type: Inner +Join condition: None -(39) CometProject +(41) Project [codegen id : 10] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, i_category#23, i_brand#24, s_store_name#25, s_company_name#26, sum_sales#31, rn#32] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -(40) ReusedExchange [Reuses operator id: 37] -Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] +(42) ReusedExchange [Reuses operator id: 34] +Output [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] + +(43) CometSort +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +Arguments: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39], [i_category#33 ASC NULLS FIRST, i_brand#34 ASC NULLS FIRST, s_store_name#35 ASC NULLS FIRST, s_company_name#36 ASC NULLS FIRST, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] + +(44) CometColumnarToRow [codegen id : 8] +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] + +(45) Window +Input [7]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39] +Arguments: [rank(d_year#37, d_moy#38) windowspecdefinition(i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#40], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36], [d_year#37 ASC NULLS FIRST, d_moy#38 ASC NULLS FIRST] + +(46) Project [codegen id : 9] +Output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +Input [8]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, d_year#37, d_moy#38, sum_sales#39, rn#40] -(41) CometBroadcastHashJoin -Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31] -Right output [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21], [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#38 - 1)], Inner, BuildRight +(47) BroadcastExchange +Input [6]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(42) CometProject -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#37, rn#38] -Arguments: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#39, sum_sales#37 AS nsum#40] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#21] +Right keys [5]: [i_category#33, i_brand#34, s_store_name#35, s_company_name#36, (rn#40 - 1)] +Join type: Inner +Join condition: None -(43) CometTakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#22,sum_sales#19,psum#39,nsum#40]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40], 100, 0, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(49) Project [codegen id : 10] +Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, sum_sales#31 AS psum#41, sum_sales#39 AS nsum#42] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#19, avg_monthly_sales#22, rn#21, sum_sales#31, i_category#33, i_brand#34, s_store_name#35, s_company_name#36, sum_sales#39, rn#40] -(44) CometColumnarToRow [codegen id : 4] -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#39, nsum#40] +(50) TakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] +Arguments: 100, [(sum_sales#19 - avg_monthly_sales#22) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#22, sum_sales#19, psum#41, nsum#42] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(52) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(47) CometColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(48) BroadcastExchange +(54) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt index eeae7c1e95..1a4201332c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/extended.txt @@ -1,78 +1,80 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, s_store_name#3, s_company_name#4, d_year#5, d_moy#6, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS sum_sales#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#7))#8,17,2) AS _w0#10)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#11, i_brand#12, s_store_name#13, s_company_name#14, d_year#15, d_moy#16, MakeDecimal(sum(UnscaledValue(ss_sales_price#17))#8,17,2) AS sum_sales#18)] @@ -103,4 +105,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt index 77ba154cab..b08d56b5ca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q47/simplified.txt @@ -1,57 +1,77 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 - WholeStageCodegen (2) - HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] + WholeStageCodegen (10) + Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (3) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (2) CometColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,s_store_name,s_company_name] #8 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] [sum(UnscaledValue(ss_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt index 2c993b48e0..74702d596d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/explain.txt @@ -1,79 +1,82 @@ == Physical Plan == -* CometColumnarToRow (75) -+- CometTakeOrderedAndProject (74) - +- CometHashAggregate (73) - +- CometExchange (72) - +- CometHashAggregate (71) - +- CometUnion (70) - :- CometProject (25) - : +- CometFilter (24) - : +- CometWindowExec (23) - : +- CometSort (22) - : +- CometWindowExec (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- CometProject (47) - : +- CometFilter (46) - : +- CometWindowExec (45) - : +- CometSort (44) - : +- CometWindowExec (43) - : +- CometSort (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometBroadcastExchange (29) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (35) - +- CometProject (69) - +- CometFilter (68) - +- CometWindowExec (67) - +- CometSort (66) - +- CometWindowExec (65) - +- CometSort (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometProject (56) - : +- CometBroadcastHashJoin (55) - : :- CometBroadcastExchange (51) - : : +- CometProject (50) - : : +- CometFilter (49) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) - +- ReusedExchange (57) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -170,27 +173,30 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) CometSort +(22) Window +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] + +(23) Sort [codegen id : 2] Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 -(23) CometWindowExec +(24) Window Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(24) CometFilter +(25) Filter [codegen id : 3] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(25) CometProject +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -198,97 +204,100 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(28) CometProject +(29) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(29) CometBroadcastExchange +(30) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(32) CometProject +(33) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(33) CometBroadcastHashJoin +(34) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(34) CometProject +(35) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(35) ReusedExchange [Reuses operator id: 13] +(36) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(36) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(37) CometProject +(38) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(38) CometHashAggregate +(39) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(39) CometExchange +(40) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) CometHashAggregate +(41) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(41) CometExchange +(42) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(42) CometSort +(43) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(43) CometWindowExec +(44) CometColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] + +(45) Window Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(44) CometSort +(46) Sort [codegen id : 5] Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 -(45) CometWindowExec +(47) Window Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(46) CometFilter +(48) Filter [codegen id : 6] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(47) CometProject +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -296,156 +305,158 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(49) CometFilter +(51) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(50) CometProject +(52) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(51) CometBroadcastExchange +(53) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(54) CometProject +(56) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(56) CometProject +(58) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(57) ReusedExchange [Reuses operator id: 13] +(59) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(59) CometProject +(61) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(60) CometHashAggregate +(62) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(61) CometExchange +(63) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(62) CometHashAggregate +(64) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(65) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(64) CometSort +(66) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(65) CometWindowExec +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(66) CometSort +(68) Window +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] + +(69) Sort [codegen id : 8] Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 -(67) CometWindowExec +(70) Window Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(68) CometFilter +(71) Filter [codegen id : 9] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(69) CometProject +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(70) CometUnion -Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(73) Union -(71) CometHashAggregate +(74) HashAggregate [codegen id : 10] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(72) CometExchange +(75) CometColumnarExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometHashAggregate +(76) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(74) CometTakeOrderedAndProject +(77) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter +(80) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(78) CometProject +(81) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(79) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(80) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt index 6a8de7fe28..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/extended.txt @@ -1,91 +1,94 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- Filter + +- Window + +- Sort + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt index 4f128e83de..ca80833ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49.native_iceberg_compat/simplified.txt @@ -1,87 +1,110 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometUnion [channel,item,return_ratio,return_rank,currency_rank] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt index 2c993b48e0..74702d596d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/explain.txt @@ -1,79 +1,82 @@ == Physical Plan == -* CometColumnarToRow (75) -+- CometTakeOrderedAndProject (74) - +- CometHashAggregate (73) - +- CometExchange (72) - +- CometHashAggregate (71) - +- CometUnion (70) - :- CometProject (25) - : +- CometFilter (24) - : +- CometWindowExec (23) - : +- CometSort (22) - : +- CometWindowExec (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- CometProject (47) - : +- CometFilter (46) - : +- CometWindowExec (45) - : +- CometSort (44) - : +- CometWindowExec (43) - : +- CometSort (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometBroadcastExchange (29) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (35) - +- CometProject (69) - +- CometFilter (68) - +- CometWindowExec (67) - +- CometSort (66) - +- CometWindowExec (65) - +- CometSort (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometProject (56) - : +- CometBroadcastHashJoin (55) - : :- CometBroadcastExchange (51) - : : +- CometProject (50) - : : +- CometFilter (49) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) - +- ReusedExchange (57) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -170,27 +173,30 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) CometSort +(22) Window +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] + +(23) Sort [codegen id : 2] Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 -(23) CometWindowExec +(24) Window Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(24) CometFilter +(25) Filter [codegen id : 3] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(25) CometProject +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -198,97 +204,100 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(28) CometProject +(29) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(29) CometBroadcastExchange +(30) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(32) CometProject +(33) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(33) CometBroadcastHashJoin +(34) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(34) CometProject +(35) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(35) ReusedExchange [Reuses operator id: 13] +(36) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(36) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(37) CometProject +(38) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(38) CometHashAggregate +(39) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(39) CometExchange +(40) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) CometHashAggregate +(41) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(41) CometExchange +(42) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(42) CometSort +(43) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(43) CometWindowExec +(44) CometColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] + +(45) Window Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(44) CometSort +(46) Sort [codegen id : 5] Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 -(45) CometWindowExec +(47) Window Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(46) CometFilter +(48) Filter [codegen id : 6] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(47) CometProject +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -296,156 +305,158 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(49) CometFilter +(51) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(50) CometProject +(52) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(51) CometBroadcastExchange +(53) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(54) CometProject +(56) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(56) CometProject +(58) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(57) ReusedExchange [Reuses operator id: 13] +(59) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(59) CometProject +(61) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(60) CometHashAggregate +(62) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(61) CometExchange +(63) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(62) CometHashAggregate +(64) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(65) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(64) CometSort +(66) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(65) CometWindowExec +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(66) CometSort +(68) Window +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] + +(69) Sort [codegen id : 8] Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 -(67) CometWindowExec +(70) Window Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(68) CometFilter +(71) Filter [codegen id : 9] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(69) CometProject +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(70) CometUnion -Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(73) Union -(71) CometHashAggregate +(74) HashAggregate [codegen id : 10] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(72) CometExchange +(75) CometColumnarExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometHashAggregate +(76) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(74) CometTakeOrderedAndProject +(77) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter +(80) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(78) CometProject +(81) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(79) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(80) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt index 6a8de7fe28..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/extended.txt @@ -1,91 +1,94 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- Filter + +- Window + +- Sort + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt index 4f128e83de..ca80833ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q49/simplified.txt @@ -1,87 +1,110 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometUnion [channel,item,return_ratio,return_rank,currency_rank] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt index 000038216f..d850c164a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/explain.txt @@ -1,75 +1,82 @@ == Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometFilter (69) - +- CometHashAggregate (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometBroadcastHashJoin (65) - :- CometWindowExec (59) - : +- CometSort (58) - : +- CometExchange (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometSortMergeJoin (54) - : :- CometSort (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometExchange (25) - : : +- CometHashAggregate (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometProject (16) - : : : +- CometWindowExec (15) - : : : +- CometSort (14) - : : : +- CometColumnarExchange (13) - : : : +- * HashAggregate (12) - : : : +- * CometColumnarToRow (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (21) - : : +- CometProject (20) - : : +- CometWindowExec (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (53) - : +- CometExchange (52) - : +- CometHashAggregate (51) - : +- CometExchange (50) - : +- CometHashAggregate (49) - : +- CometProject (48) - : +- CometBroadcastHashJoin (47) - : :- CometProject (41) - : : +- CometWindowExec (40) - : : +- CometSort (39) - : : +- CometColumnarExchange (38) - : : +- * HashAggregate (37) - : : +- * CometColumnarToRow (36) - : : +- CometExchange (35) - : : +- CometHashAggregate (34) - : : +- CometProject (33) - : : +- CometBroadcastHashJoin (32) - : : :- CometFilter (30) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (29) - : : +- ReusedExchange (31) - : +- CometBroadcastExchange (46) - : +- CometProject (45) - : +- CometWindowExec (44) - : +- CometSort (43) - : +- ReusedExchange (42) - +- CometBroadcastExchange (64) - +- CometProject (63) - +- CometWindowExec (62) - +- CometSort (61) - +- ReusedExchange (60) +TakeOrderedAndProject (78) ++- * Filter (77) + +- * HashAggregate (76) + +- * HashAggregate (75) + +- * Project (74) + +- * BroadcastHashJoin Inner BuildRight (73) + :- Window (66) + : +- * CometColumnarToRow (65) + : +- CometSort (64) + : +- CometExchange (63) + : +- CometProject (62) + : +- CometFilter (61) + : +- CometSortMergeJoin (60) + : :- CometSort (31) + : : +- CometColumnarExchange (30) + : : +- * HashAggregate (29) + : : +- * CometColumnarToRow (28) + : : +- CometColumnarExchange (27) + : : +- * HashAggregate (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (17) + : : : +- Window (16) + : : : +- * CometColumnarToRow (15) + : : : +- CometSort (14) + : : : +- CometColumnarExchange (13) + : : : +- * HashAggregate (12) + : : : +- * CometColumnarToRow (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- Window (21) + : : +- * CometColumnarToRow (20) + : : +- CometSort (19) + : : +- ReusedExchange (18) + : +- CometSort (59) + : +- CometColumnarExchange (58) + : +- * HashAggregate (57) + : +- * CometColumnarToRow (56) + : +- CometColumnarExchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * Project (45) + : : +- Window (44) + : : +- * CometColumnarToRow (43) + : : +- CometSort (42) + : : +- CometColumnarExchange (41) + : : +- * HashAggregate (40) + : : +- * CometColumnarToRow (39) + : : +- CometExchange (38) + : : +- CometHashAggregate (37) + : : +- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometFilter (33) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (32) + : : +- ReusedExchange (34) + : +- BroadcastExchange (51) + : +- * Project (50) + : +- Window (49) + : +- * CometColumnarToRow (48) + : +- CometSort (47) + : +- ReusedExchange (46) + +- BroadcastExchange (72) + +- * Project (71) + +- Window (70) + +- * CometColumnarToRow (69) + +- CometSort (68) + +- ReusedExchange (67) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -139,279 +146,315 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) CometWindowExec +(15) CometColumnarToRow [codegen id : 2] Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12], [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(16) CometProject +(16) Window +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] + +(17) Project [codegen id : 6] +Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] -Arguments: [item_sk#10, d_date#6, sumws#11, rk#12], [item_sk#10, d_date#6, sumws#11, rk#12] -(17) ReusedExchange [Reuses operator id: 13] +(18) ReusedExchange [Reuses operator id: 13] Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -(18) CometSort +(19) CometSort Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16], [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST] -(19) CometWindowExec +(20) CometColumnarToRow [codegen id : 4] +Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] + +(21) Window Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17], [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] +Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] -(20) CometProject +(22) Project [codegen id : 5] +Output [3]: [item_sk#13, sumws#15, rk#17] Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] -Arguments: [item_sk#13, sumws#15, rk#17], [item_sk#13, sumws#15, rk#17] -(21) CometBroadcastExchange +(23) BroadcastExchange Input [3]: [item_sk#13, sumws#15, rk#17] -Arguments: [item_sk#13, sumws#15, rk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) CometBroadcastHashJoin -Left output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] -Right output [3]: [item_sk#13, sumws#15, rk#17] -Arguments: [item_sk#10], [item_sk#13], Inner, (rk#12 >= rk#17), BuildRight +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [item_sk#10] +Right keys [1]: [item_sk#13] +Join type: Inner +Join condition: (rk#12 >= rk#17) -(23) CometProject +(25) Project [codegen id : 6] +Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] -Arguments: [item_sk#10, d_date#6, sumws#11, sumws#15], [item_sk#10, d_date#6, sumws#11, sumws#15] -(24) CometHashAggregate +(26) HashAggregate [codegen id : 6] Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [partial_sum(sumws#15)] +Aggregate Attributes [2]: [sum#18, isEmpty#19] +Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(25) CometExchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#18, isEmpty#19] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(27) CometColumnarExchange +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) CometHashAggregate -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#18, isEmpty#19] +(28) CometColumnarToRow [codegen id : 7] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] + +(29) HashAggregate [codegen id : 7] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [sum(sumws#15)] +Aggregate Attributes [1]: [sum(sumws#15)#22] +Results [3]: [item_sk#10, d_date#6, sum(sumws#15)#22 AS cume_sales#23] -(27) CometExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#20] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(30) CometColumnarExchange +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(28) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#20] -Arguments: [item_sk#10, d_date#6, cume_sales#20], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(31) CometSort +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(30) CometFilter -Input [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_item_sk#21) - -(31) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#25, d_date#26] - -(32) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_date#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(33) CometProject -Input [5]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23, d_date_sk#25, d_date#26] -Arguments: [ss_item_sk#21, ss_sales_price#22, d_date#26], [ss_item_sk#21, ss_sales_price#22, d_date#26] - -(34) CometHashAggregate -Input [3]: [ss_item_sk#21, ss_sales_price#22, d_date#26] -Keys [2]: [ss_item_sk#21, d_date#26] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#22))] - -(35) CometExchange -Input [3]: [ss_item_sk#21, d_date#26, sum#27] -Arguments: hashpartitioning(ss_item_sk#21, d_date#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(36) CometColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#21, d_date#26, sum#27] - -(37) HashAggregate [codegen id : 3] -Input [3]: [ss_item_sk#21, d_date#26, sum#27] -Keys [2]: [ss_item_sk#21, d_date#26] -Functions [1]: [sum(UnscaledValue(ss_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#22))#28] -Results [4]: [ss_item_sk#21 AS item_sk#29, d_date#26, MakeDecimal(sum(UnscaledValue(ss_sales_price#22))#28,17,2) AS sumss#30, ss_item_sk#21] - -(38) CometColumnarExchange -Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] -Arguments: hashpartitioning(ss_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(39) CometSort -Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] -Arguments: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21], [ss_item_sk#21 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST] - -(40) CometWindowExec -Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] -Arguments: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21, rk#31], [row_number() windowspecdefinition(ss_item_sk#21, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#31], [ss_item_sk#21], [d_date#26 ASC NULLS FIRST] - -(41) CometProject -Input [5]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21, rk#31] -Arguments: [item_sk#29, d_date#26, sumss#30, rk#31], [item_sk#29, d_date#26, sumss#30, rk#31] - -(42) ReusedExchange [Reuses operator id: 38] -Output [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] - -(43) CometSort -Input [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] -Arguments: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35], [ss_item_sk#35 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] - -(44) CometWindowExec -Input [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] -Arguments: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35, rk#36], [row_number() windowspecdefinition(ss_item_sk#35, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [ss_item_sk#35], [d_date#33 ASC NULLS FIRST] - -(45) CometProject -Input [5]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35, rk#36] -Arguments: [item_sk#32, sumss#34, rk#36], [item_sk#32, sumss#34, rk#36] - -(46) CometBroadcastExchange -Input [3]: [item_sk#32, sumss#34, rk#36] -Arguments: [item_sk#32, sumss#34, rk#36] - -(47) CometBroadcastHashJoin -Left output [4]: [item_sk#29, d_date#26, sumss#30, rk#31] -Right output [3]: [item_sk#32, sumss#34, rk#36] -Arguments: [item_sk#29], [item_sk#32], Inner, (rk#31 >= rk#36), BuildRight - -(48) CometProject -Input [7]: [item_sk#29, d_date#26, sumss#30, rk#31, item_sk#32, sumss#34, rk#36] -Arguments: [item_sk#29, d_date#26, sumss#30, sumss#34], [item_sk#29, d_date#26, sumss#30, sumss#34] - -(49) CometHashAggregate -Input [4]: [item_sk#29, d_date#26, sumss#30, sumss#34] -Keys [3]: [item_sk#29, d_date#26, sumss#30] -Functions [1]: [partial_sum(sumss#34)] - -(50) CometExchange -Input [5]: [item_sk#29, d_date#26, sumss#30, sum#37, isEmpty#38] -Arguments: hashpartitioning(item_sk#29, d_date#26, sumss#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(51) CometHashAggregate -Input [5]: [item_sk#29, d_date#26, sumss#30, sum#37, isEmpty#38] -Keys [3]: [item_sk#29, d_date#26, sumss#30] -Functions [1]: [sum(sumss#34)] - -(52) CometExchange -Input [3]: [item_sk#29, d_date#26, cume_sales#39] -Arguments: hashpartitioning(item_sk#29, d_date#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(53) CometSort -Input [3]: [item_sk#29, d_date#26, cume_sales#39] -Arguments: [item_sk#29, d_date#26, cume_sales#39], [item_sk#29 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST] - -(54) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#20] -Right output [3]: [item_sk#29, d_date#26, cume_sales#39] -Arguments: [item_sk#10, d_date#6], [item_sk#29, d_date#26], FullOuter - -(55) CometFilter -Input [6]: [item_sk#10, d_date#6, cume_sales#20, item_sk#29, d_date#26, cume_sales#39] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#29 END) - -(56) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#20, item_sk#29, d_date#26, cume_sales#39] -Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#29 END AS item_sk#40, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#26 END AS d_date#41, cume_sales#20 AS web_sales#42, cume_sales#39 AS store_sales#43] - -(57) CometExchange -Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] -Arguments: hashpartitioning(item_sk#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(58) CometSort -Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] -Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43], [item_sk#40 ASC NULLS FIRST, d_date#41 ASC NULLS FIRST] - -(59) CometWindowExec -Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] -Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44], [row_number() windowspecdefinition(item_sk#40, d_date#41 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#44], [item_sk#40], [d_date#41 ASC NULLS FIRST] - -(60) ReusedExchange [Reuses operator id: 57] -Output [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] - -(61) CometSort -Input [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] -Arguments: [item_sk#45, d_date#46, web_sales#47, store_sales#48], [item_sk#45 ASC NULLS FIRST, d_date#46 ASC NULLS FIRST] - -(62) CometWindowExec -Input [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] -Arguments: [item_sk#45, d_date#46, web_sales#47, store_sales#48, rk#49], [row_number() windowspecdefinition(item_sk#45, d_date#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#45], [d_date#46 ASC NULLS FIRST] +(33) CometFilter +Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_item_sk#24) + +(34) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#28, d_date#29] + +(35) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Right output [2]: [d_date_sk#28, d_date#29] +Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight + +(36) CometProject +Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] +Arguments: [ss_item_sk#24, ss_sales_price#25, d_date#29], [ss_item_sk#24, ss_sales_price#25, d_date#29] + +(37) CometHashAggregate +Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] + +(38) CometExchange +Input [3]: [ss_item_sk#24, d_date#29, sum#30] +Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(39) CometColumnarToRow [codegen id : 8] +Input [3]: [ss_item_sk#24, d_date#29, sum#30] + +(40) HashAggregate [codegen id : 8] +Input [3]: [ss_item_sk#24, d_date#29, sum#30] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] +Results [4]: [ss_item_sk#24 AS item_sk#32, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS sumss#33, ss_item_sk#24] + +(41) CometColumnarExchange +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(42) CometSort +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +Arguments: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] + +(43) CometColumnarToRow [codegen id : 9] +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -(63) CometProject -Input [5]: [item_sk#45, d_date#46, web_sales#47, store_sales#48, rk#49] -Arguments: [item_sk#45, web_sales#47, store_sales#48, rk#49], [item_sk#45, web_sales#47, store_sales#48, rk#49] +(44) Window +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] -(64) CometBroadcastExchange -Input [4]: [item_sk#45, web_sales#47, store_sales#48, rk#49] -Arguments: [item_sk#45, web_sales#47, store_sales#48, rk#49] +(45) Project [codegen id : 13] +Output [4]: [item_sk#32, d_date#29, sumss#33, rk#34] +Input [5]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24, rk#34] -(65) CometBroadcastHashJoin -Left output [5]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44] -Right output [4]: [item_sk#45, web_sales#47, store_sales#48, rk#49] -Arguments: [item_sk#40], [item_sk#45], Inner, (rk#44 >= rk#49), BuildRight +(46) ReusedExchange [Reuses operator id: 41] +Output [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -(66) CometProject -Input [9]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44, item_sk#45, web_sales#47, store_sales#48, rk#49] -Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48], [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48] +(47) CometSort +Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +Arguments: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38], [ss_item_sk#38 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] -(67) CometHashAggregate -Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48] -Keys [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] -Functions [2]: [partial_max(web_sales#47), partial_max(store_sales#48)] - -(68) CometHashAggregate -Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, max#50, max#51] -Keys [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] -Functions [2]: [max(web_sales#47), max(store_sales#48)] - -(69) CometFilter -Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] -Condition : ((isnotnull(web_cumulative#52) AND isnotnull(store_cumulative#53)) AND (web_cumulative#52 > store_cumulative#53)) - -(70) CometTakeOrderedAndProject -Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#40 ASC NULLS FIRST,d_date#41 ASC NULLS FIRST], output=[item_sk#40,d_date#41,web_sales#42,store_sales#43,web_cumulative#52,store_cumulative#53]), [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53], 100, 0, [item_sk#40 ASC NULLS FIRST, d_date#41 ASC NULLS FIRST], [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] - -(71) CometColumnarToRow [codegen id : 9] -Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] +(48) CometColumnarToRow [codegen id : 11] +Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] + +(49) Window +Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +Arguments: [row_number() windowspecdefinition(ss_item_sk#38, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#39], [ss_item_sk#38], [d_date#36 ASC NULLS FIRST] + +(50) Project [codegen id : 12] +Output [3]: [item_sk#35, sumss#37, rk#39] +Input [5]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38, rk#39] + +(51) BroadcastExchange +Input [3]: [item_sk#35, sumss#37, rk#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +(52) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [item_sk#32] +Right keys [1]: [item_sk#35] +Join type: Inner +Join condition: (rk#34 >= rk#39) + +(53) Project [codegen id : 13] +Output [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] +Input [7]: [item_sk#32, d_date#29, sumss#33, rk#34, item_sk#35, sumss#37, rk#39] + +(54) HashAggregate [codegen id : 13] +Input [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] +Keys [3]: [item_sk#32, d_date#29, sumss#33] +Functions [1]: [partial_sum(sumss#37)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] + +(55) CometColumnarExchange +Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +Arguments: hashpartitioning(item_sk#32, d_date#29, sumss#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(56) CometColumnarToRow [codegen id : 14] +Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] + +(57) HashAggregate [codegen id : 14] +Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +Keys [3]: [item_sk#32, d_date#29, sumss#33] +Functions [1]: [sum(sumss#37)] +Aggregate Attributes [1]: [sum(sumss#37)#44] +Results [3]: [item_sk#32, d_date#29, sum(sumss#37)#44 AS cume_sales#45] + +(58) CometColumnarExchange +Input [3]: [item_sk#32, d_date#29, cume_sales#45] +Arguments: hashpartitioning(item_sk#32, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(59) CometSort +Input [3]: [item_sk#32, d_date#29, cume_sales#45] +Arguments: [item_sk#32, d_date#29, cume_sales#45], [item_sk#32 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] + +(60) CometSortMergeJoin +Left output [3]: [item_sk#10, d_date#6, cume_sales#23] +Right output [3]: [item_sk#32, d_date#29, cume_sales#45] +Arguments: [item_sk#10, d_date#6], [item_sk#32, d_date#29], FullOuter + +(61) CometFilter +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END) + +(62) CometProject +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] +Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END AS item_sk#46, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#47, cume_sales#23 AS web_sales#48, cume_sales#45 AS store_sales#49] + +(63) CometExchange +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(64) CometSort +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] + +(65) CometColumnarToRow [codegen id : 15] +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] + +(66) Window +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 ASC NULLS FIRST] + +(67) ReusedExchange [Reuses operator id: 63] +Output [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] + +(68) CometSort +Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +Arguments: [item_sk#51, d_date#52, web_sales#53, store_sales#54], [item_sk#51 ASC NULLS FIRST, d_date#52 ASC NULLS FIRST] + +(69) CometColumnarToRow [codegen id : 30] +Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] + +(70) Window +Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +Arguments: [row_number() windowspecdefinition(item_sk#51, d_date#52 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#55], [item_sk#51], [d_date#52 ASC NULLS FIRST] + +(71) Project [codegen id : 31] +Output [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] +Input [5]: [item_sk#51, d_date#52, web_sales#53, store_sales#54, rk#55] + +(72) BroadcastExchange +Input [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] + +(73) BroadcastHashJoin [codegen id : 32] +Left keys [1]: [item_sk#46] +Right keys [1]: [item_sk#51] +Join type: Inner +Join condition: (rk#50 >= rk#55) + +(74) Project [codegen id : 32] +Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] +Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#51, web_sales#53, store_sales#54, rk#55] + +(75) HashAggregate [codegen id : 32] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] +Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] +Aggregate Attributes [2]: [max#56, max#57] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] + +(76) HashAggregate [codegen id : 32] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] +Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Functions [2]: [max(web_sales#53), max(store_sales#54)] +Aggregate Attributes [2]: [max(web_sales#53)#60, max(store_sales#54)#61] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#60 AS web_cumulative#62, max(store_sales#54)#61 AS store_cumulative#63] + +(77) Filter [codegen id : 32] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) + +(78) TakeOrderedAndProject +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(80) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(74) CometProject +(81) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(75) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(76) BroadcastExchange +(83) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt index 6e3fd36930..f7546ea154 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/extended.txt @@ -1,211 +1,224 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometWindowExec - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- HashAggregate + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- Window [COMET: Window expressions are not supported] + : : : +- CometColumnarToRow + : : : +- CometSort + : : : +- CometColumnarExchange + : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : : +- CometColumnarToRow + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 176 out of 196 eligible operators (89%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 130 out of 196 eligible operators (66%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt index 51d1e38ec6..c88883dbad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a.native_iceberg_compat/simplified.txt @@ -1,86 +1,122 @@ -WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometHashAggregate [max,max] [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative,max(web_sales),max(store_sales)] - CometHashAggregate [web_sales,store_sales] [item_sk,d_date,web_sales,store_sales,max,max] - CometProject [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - CometBroadcastHashJoin [item_sk,d_date,web_sales,store_sales,rk,item_sk,web_sales,store_sales,rk] - CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #2 - CometHashAggregate [sumws,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumws)] - CometExchange [item_sk,d_date,sumws] #3 - CometHashAggregate [sumws] [item_sk,d_date,sumws,sum,isEmpty] - CometProject [item_sk,d_date,sumws,sumws] - CometBroadcastHashJoin [item_sk,d_date,sumws,rk,item_sk,sumws,rk] - CometProject [item_sk,d_date,sumws,rk] - CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (32) + Filter [web_cumulative,store_cumulative] + HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (15) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (7) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (6) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] InputAdapter - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [item_sk,sumws,rk] #8 - CometProject [item_sk,sumws,rk] - CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #9 - CometHashAggregate [sumss,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumss)] - CometExchange [item_sk,d_date,sumss] #10 - CometHashAggregate [sumss] [item_sk,d_date,sumss,sum,isEmpty] - CometProject [item_sk,d_date,sumss,sumss] - CometBroadcastHashJoin [item_sk,d_date,sumss,rk,item_sk,sumss,rk] - CometProject [item_sk,d_date,sumss,rk] - CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #11 - WholeStageCodegen (3) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow + Window [ws_item_sk,d_date] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + CometColumnarToRow + InputAdapter + CometExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #9 + WholeStageCodegen (14) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + WholeStageCodegen (13) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] InputAdapter - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 - CometBroadcastExchange [item_sk,sumss,rk] #13 - CometProject [item_sk,sumss,rk] - CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - CometBroadcastExchange [item_sk,web_sales,store_sales,rk] #14 - CometProject [item_sk,web_sales,store_sales,rk] - CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + Window [ss_item_sk,d_date] + WholeStageCodegen (9) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #11 + WholeStageCodegen (8) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + CometColumnarToRow + InputAdapter + CometExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (11) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (31) + Project [item_sk,web_sales,store_sales,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (30) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt index 000038216f..d850c164a2 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/explain.txt @@ -1,75 +1,82 @@ == Physical Plan == -* CometColumnarToRow (71) -+- CometTakeOrderedAndProject (70) - +- CometFilter (69) - +- CometHashAggregate (68) - +- CometHashAggregate (67) - +- CometProject (66) - +- CometBroadcastHashJoin (65) - :- CometWindowExec (59) - : +- CometSort (58) - : +- CometExchange (57) - : +- CometProject (56) - : +- CometFilter (55) - : +- CometSortMergeJoin (54) - : :- CometSort (28) - : : +- CometExchange (27) - : : +- CometHashAggregate (26) - : : +- CometExchange (25) - : : +- CometHashAggregate (24) - : : +- CometProject (23) - : : +- CometBroadcastHashJoin (22) - : : :- CometProject (16) - : : : +- CometWindowExec (15) - : : : +- CometSort (14) - : : : +- CometColumnarExchange (13) - : : : +- * HashAggregate (12) - : : : +- * CometColumnarToRow (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (21) - : : +- CometProject (20) - : : +- CometWindowExec (19) - : : +- CometSort (18) - : : +- ReusedExchange (17) - : +- CometSort (53) - : +- CometExchange (52) - : +- CometHashAggregate (51) - : +- CometExchange (50) - : +- CometHashAggregate (49) - : +- CometProject (48) - : +- CometBroadcastHashJoin (47) - : :- CometProject (41) - : : +- CometWindowExec (40) - : : +- CometSort (39) - : : +- CometColumnarExchange (38) - : : +- * HashAggregate (37) - : : +- * CometColumnarToRow (36) - : : +- CometExchange (35) - : : +- CometHashAggregate (34) - : : +- CometProject (33) - : : +- CometBroadcastHashJoin (32) - : : :- CometFilter (30) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (29) - : : +- ReusedExchange (31) - : +- CometBroadcastExchange (46) - : +- CometProject (45) - : +- CometWindowExec (44) - : +- CometSort (43) - : +- ReusedExchange (42) - +- CometBroadcastExchange (64) - +- CometProject (63) - +- CometWindowExec (62) - +- CometSort (61) - +- ReusedExchange (60) +TakeOrderedAndProject (78) ++- * Filter (77) + +- * HashAggregate (76) + +- * HashAggregate (75) + +- * Project (74) + +- * BroadcastHashJoin Inner BuildRight (73) + :- Window (66) + : +- * CometColumnarToRow (65) + : +- CometSort (64) + : +- CometExchange (63) + : +- CometProject (62) + : +- CometFilter (61) + : +- CometSortMergeJoin (60) + : :- CometSort (31) + : : +- CometColumnarExchange (30) + : : +- * HashAggregate (29) + : : +- * CometColumnarToRow (28) + : : +- CometColumnarExchange (27) + : : +- * HashAggregate (26) + : : +- * Project (25) + : : +- * BroadcastHashJoin Inner BuildRight (24) + : : :- * Project (17) + : : : +- Window (16) + : : : +- * CometColumnarToRow (15) + : : : +- CometSort (14) + : : : +- CometColumnarExchange (13) + : : : +- * HashAggregate (12) + : : : +- * CometColumnarToRow (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (23) + : : +- * Project (22) + : : +- Window (21) + : : +- * CometColumnarToRow (20) + : : +- CometSort (19) + : : +- ReusedExchange (18) + : +- CometSort (59) + : +- CometColumnarExchange (58) + : +- * HashAggregate (57) + : +- * CometColumnarToRow (56) + : +- CometColumnarExchange (55) + : +- * HashAggregate (54) + : +- * Project (53) + : +- * BroadcastHashJoin Inner BuildRight (52) + : :- * Project (45) + : : +- Window (44) + : : +- * CometColumnarToRow (43) + : : +- CometSort (42) + : : +- CometColumnarExchange (41) + : : +- * HashAggregate (40) + : : +- * CometColumnarToRow (39) + : : +- CometExchange (38) + : : +- CometHashAggregate (37) + : : +- CometProject (36) + : : +- CometBroadcastHashJoin (35) + : : :- CometFilter (33) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (32) + : : +- ReusedExchange (34) + : +- BroadcastExchange (51) + : +- * Project (50) + : +- Window (49) + : +- * CometColumnarToRow (48) + : +- CometSort (47) + : +- ReusedExchange (46) + +- BroadcastExchange (72) + +- * Project (71) + +- Window (70) + +- * CometColumnarToRow (69) + +- CometSort (68) + +- ReusedExchange (67) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -139,279 +146,315 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometColumnar Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(15) CometWindowExec +(15) CometColumnarToRow [codegen id : 2] Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] -Arguments: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12], [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(16) CometProject +(16) Window +Input [4]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#12], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] + +(17) Project [codegen id : 6] +Output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] Input [5]: [item_sk#10, d_date#6, sumws#11, ws_item_sk#1, rk#12] -Arguments: [item_sk#10, d_date#6, sumws#11, rk#12], [item_sk#10, d_date#6, sumws#11, rk#12] -(17) ReusedExchange [Reuses operator id: 13] +(18) ReusedExchange [Reuses operator id: 13] Output [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -(18) CometSort +(19) CometSort Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16], [ws_item_sk#16 ASC NULLS FIRST, d_date#14 ASC NULLS FIRST] -(19) CometWindowExec +(20) CometColumnarToRow [codegen id : 4] +Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] + +(21) Window Input [4]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16] -Arguments: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17], [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] +Arguments: [row_number() windowspecdefinition(ws_item_sk#16, d_date#14 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#17], [ws_item_sk#16], [d_date#14 ASC NULLS FIRST] -(20) CometProject +(22) Project [codegen id : 5] +Output [3]: [item_sk#13, sumws#15, rk#17] Input [5]: [item_sk#13, d_date#14, sumws#15, ws_item_sk#16, rk#17] -Arguments: [item_sk#13, sumws#15, rk#17], [item_sk#13, sumws#15, rk#17] -(21) CometBroadcastExchange +(23) BroadcastExchange Input [3]: [item_sk#13, sumws#15, rk#17] -Arguments: [item_sk#13, sumws#15, rk#17] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(22) CometBroadcastHashJoin -Left output [4]: [item_sk#10, d_date#6, sumws#11, rk#12] -Right output [3]: [item_sk#13, sumws#15, rk#17] -Arguments: [item_sk#10], [item_sk#13], Inner, (rk#12 >= rk#17), BuildRight +(24) BroadcastHashJoin [codegen id : 6] +Left keys [1]: [item_sk#10] +Right keys [1]: [item_sk#13] +Join type: Inner +Join condition: (rk#12 >= rk#17) -(23) CometProject +(25) Project [codegen id : 6] +Output [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] Input [7]: [item_sk#10, d_date#6, sumws#11, rk#12, item_sk#13, sumws#15, rk#17] -Arguments: [item_sk#10, d_date#6, sumws#11, sumws#15], [item_sk#10, d_date#6, sumws#11, sumws#15] -(24) CometHashAggregate +(26) HashAggregate [codegen id : 6] Input [4]: [item_sk#10, d_date#6, sumws#11, sumws#15] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [partial_sum(sumws#15)] +Aggregate Attributes [2]: [sum#18, isEmpty#19] +Results [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] -(25) CometExchange -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#18, isEmpty#19] -Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(27) CometColumnarExchange +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] +Arguments: hashpartitioning(item_sk#10, d_date#6, sumws#11, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(26) CometHashAggregate -Input [5]: [item_sk#10, d_date#6, sumws#11, sum#18, isEmpty#19] +(28) CometColumnarToRow [codegen id : 7] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] + +(29) HashAggregate [codegen id : 7] +Input [5]: [item_sk#10, d_date#6, sumws#11, sum#20, isEmpty#21] Keys [3]: [item_sk#10, d_date#6, sumws#11] Functions [1]: [sum(sumws#15)] +Aggregate Attributes [1]: [sum(sumws#15)#22] +Results [3]: [item_sk#10, d_date#6, sum(sumws#15)#22 AS cume_sales#23] -(27) CometExchange -Input [3]: [item_sk#10, d_date#6, cume_sales#20] -Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(30) CometColumnarExchange +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: hashpartitioning(item_sk#10, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(28) CometSort -Input [3]: [item_sk#10, d_date#6, cume_sales#20] -Arguments: [item_sk#10, d_date#6, cume_sales#20], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(31) CometSort +Input [3]: [item_sk#10, d_date#6, cume_sales#23] +Arguments: [item_sk#10, d_date#6, cume_sales#23], [item_sk#10 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(29) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] +(32) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#23), dynamicpruningexpression(ss_sold_date_sk#23 IN dynamicpruning#24)] +PartitionFilters: [isnotnull(ss_sold_date_sk#26), dynamicpruningexpression(ss_sold_date_sk#26 IN dynamicpruning#27)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(30) CometFilter -Input [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] -Condition : isnotnull(ss_item_sk#21) - -(31) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#25, d_date#26] - -(32) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23] -Right output [2]: [d_date_sk#25, d_date#26] -Arguments: [ss_sold_date_sk#23], [d_date_sk#25], Inner, BuildRight - -(33) CometProject -Input [5]: [ss_item_sk#21, ss_sales_price#22, ss_sold_date_sk#23, d_date_sk#25, d_date#26] -Arguments: [ss_item_sk#21, ss_sales_price#22, d_date#26], [ss_item_sk#21, ss_sales_price#22, d_date#26] - -(34) CometHashAggregate -Input [3]: [ss_item_sk#21, ss_sales_price#22, d_date#26] -Keys [2]: [ss_item_sk#21, d_date#26] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#22))] - -(35) CometExchange -Input [3]: [ss_item_sk#21, d_date#26, sum#27] -Arguments: hashpartitioning(ss_item_sk#21, d_date#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(36) CometColumnarToRow [codegen id : 3] -Input [3]: [ss_item_sk#21, d_date#26, sum#27] - -(37) HashAggregate [codegen id : 3] -Input [3]: [ss_item_sk#21, d_date#26, sum#27] -Keys [2]: [ss_item_sk#21, d_date#26] -Functions [1]: [sum(UnscaledValue(ss_sales_price#22))] -Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#22))#28] -Results [4]: [ss_item_sk#21 AS item_sk#29, d_date#26, MakeDecimal(sum(UnscaledValue(ss_sales_price#22))#28,17,2) AS sumss#30, ss_item_sk#21] - -(38) CometColumnarExchange -Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] -Arguments: hashpartitioning(ss_item_sk#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] - -(39) CometSort -Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] -Arguments: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21], [ss_item_sk#21 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST] - -(40) CometWindowExec -Input [4]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21] -Arguments: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21, rk#31], [row_number() windowspecdefinition(ss_item_sk#21, d_date#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#31], [ss_item_sk#21], [d_date#26 ASC NULLS FIRST] - -(41) CometProject -Input [5]: [item_sk#29, d_date#26, sumss#30, ss_item_sk#21, rk#31] -Arguments: [item_sk#29, d_date#26, sumss#30, rk#31], [item_sk#29, d_date#26, sumss#30, rk#31] - -(42) ReusedExchange [Reuses operator id: 38] -Output [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] - -(43) CometSort -Input [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] -Arguments: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35], [ss_item_sk#35 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] - -(44) CometWindowExec -Input [4]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35] -Arguments: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35, rk#36], [row_number() windowspecdefinition(ss_item_sk#35, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#36], [ss_item_sk#35], [d_date#33 ASC NULLS FIRST] - -(45) CometProject -Input [5]: [item_sk#32, d_date#33, sumss#34, ss_item_sk#35, rk#36] -Arguments: [item_sk#32, sumss#34, rk#36], [item_sk#32, sumss#34, rk#36] - -(46) CometBroadcastExchange -Input [3]: [item_sk#32, sumss#34, rk#36] -Arguments: [item_sk#32, sumss#34, rk#36] - -(47) CometBroadcastHashJoin -Left output [4]: [item_sk#29, d_date#26, sumss#30, rk#31] -Right output [3]: [item_sk#32, sumss#34, rk#36] -Arguments: [item_sk#29], [item_sk#32], Inner, (rk#31 >= rk#36), BuildRight - -(48) CometProject -Input [7]: [item_sk#29, d_date#26, sumss#30, rk#31, item_sk#32, sumss#34, rk#36] -Arguments: [item_sk#29, d_date#26, sumss#30, sumss#34], [item_sk#29, d_date#26, sumss#30, sumss#34] - -(49) CometHashAggregate -Input [4]: [item_sk#29, d_date#26, sumss#30, sumss#34] -Keys [3]: [item_sk#29, d_date#26, sumss#30] -Functions [1]: [partial_sum(sumss#34)] - -(50) CometExchange -Input [5]: [item_sk#29, d_date#26, sumss#30, sum#37, isEmpty#38] -Arguments: hashpartitioning(item_sk#29, d_date#26, sumss#30, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(51) CometHashAggregate -Input [5]: [item_sk#29, d_date#26, sumss#30, sum#37, isEmpty#38] -Keys [3]: [item_sk#29, d_date#26, sumss#30] -Functions [1]: [sum(sumss#34)] - -(52) CometExchange -Input [3]: [item_sk#29, d_date#26, cume_sales#39] -Arguments: hashpartitioning(item_sk#29, d_date#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(53) CometSort -Input [3]: [item_sk#29, d_date#26, cume_sales#39] -Arguments: [item_sk#29, d_date#26, cume_sales#39], [item_sk#29 ASC NULLS FIRST, d_date#26 ASC NULLS FIRST] - -(54) CometSortMergeJoin -Left output [3]: [item_sk#10, d_date#6, cume_sales#20] -Right output [3]: [item_sk#29, d_date#26, cume_sales#39] -Arguments: [item_sk#10, d_date#6], [item_sk#29, d_date#26], FullOuter - -(55) CometFilter -Input [6]: [item_sk#10, d_date#6, cume_sales#20, item_sk#29, d_date#26, cume_sales#39] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#29 END) - -(56) CometProject -Input [6]: [item_sk#10, d_date#6, cume_sales#20, item_sk#29, d_date#26, cume_sales#39] -Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#29 END AS item_sk#40, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#26 END AS d_date#41, cume_sales#20 AS web_sales#42, cume_sales#39 AS store_sales#43] - -(57) CometExchange -Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] -Arguments: hashpartitioning(item_sk#40, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(58) CometSort -Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] -Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43], [item_sk#40 ASC NULLS FIRST, d_date#41 ASC NULLS FIRST] - -(59) CometWindowExec -Input [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] -Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44], [row_number() windowspecdefinition(item_sk#40, d_date#41 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#44], [item_sk#40], [d_date#41 ASC NULLS FIRST] - -(60) ReusedExchange [Reuses operator id: 57] -Output [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] - -(61) CometSort -Input [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] -Arguments: [item_sk#45, d_date#46, web_sales#47, store_sales#48], [item_sk#45 ASC NULLS FIRST, d_date#46 ASC NULLS FIRST] - -(62) CometWindowExec -Input [4]: [item_sk#45, d_date#46, web_sales#47, store_sales#48] -Arguments: [item_sk#45, d_date#46, web_sales#47, store_sales#48, rk#49], [row_number() windowspecdefinition(item_sk#45, d_date#46 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#45], [d_date#46 ASC NULLS FIRST] +(33) CometFilter +Input [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Condition : isnotnull(ss_item_sk#24) + +(34) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#28, d_date#29] + +(35) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26] +Right output [2]: [d_date_sk#28, d_date#29] +Arguments: [ss_sold_date_sk#26], [d_date_sk#28], Inner, BuildRight + +(36) CometProject +Input [5]: [ss_item_sk#24, ss_sales_price#25, ss_sold_date_sk#26, d_date_sk#28, d_date#29] +Arguments: [ss_item_sk#24, ss_sales_price#25, d_date#29], [ss_item_sk#24, ss_sales_price#25, d_date#29] + +(37) CometHashAggregate +Input [3]: [ss_item_sk#24, ss_sales_price#25, d_date#29] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#25))] + +(38) CometExchange +Input [3]: [ss_item_sk#24, d_date#29, sum#30] +Arguments: hashpartitioning(ss_item_sk#24, d_date#29, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(39) CometColumnarToRow [codegen id : 8] +Input [3]: [ss_item_sk#24, d_date#29, sum#30] + +(40) HashAggregate [codegen id : 8] +Input [3]: [ss_item_sk#24, d_date#29, sum#30] +Keys [2]: [ss_item_sk#24, d_date#29] +Functions [1]: [sum(UnscaledValue(ss_sales_price#25))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_sales_price#25))#31] +Results [4]: [ss_item_sk#24 AS item_sk#32, d_date#29, MakeDecimal(sum(UnscaledValue(ss_sales_price#25))#31,17,2) AS sumss#33, ss_item_sk#24] + +(41) CometColumnarExchange +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +Arguments: hashpartitioning(ss_item_sk#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] + +(42) CometSort +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +Arguments: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24], [ss_item_sk#24 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] + +(43) CometColumnarToRow [codegen id : 9] +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] -(63) CometProject -Input [5]: [item_sk#45, d_date#46, web_sales#47, store_sales#48, rk#49] -Arguments: [item_sk#45, web_sales#47, store_sales#48, rk#49], [item_sk#45, web_sales#47, store_sales#48, rk#49] +(44) Window +Input [4]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24] +Arguments: [row_number() windowspecdefinition(ss_item_sk#24, d_date#29 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#34], [ss_item_sk#24], [d_date#29 ASC NULLS FIRST] -(64) CometBroadcastExchange -Input [4]: [item_sk#45, web_sales#47, store_sales#48, rk#49] -Arguments: [item_sk#45, web_sales#47, store_sales#48, rk#49] +(45) Project [codegen id : 13] +Output [4]: [item_sk#32, d_date#29, sumss#33, rk#34] +Input [5]: [item_sk#32, d_date#29, sumss#33, ss_item_sk#24, rk#34] -(65) CometBroadcastHashJoin -Left output [5]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44] -Right output [4]: [item_sk#45, web_sales#47, store_sales#48, rk#49] -Arguments: [item_sk#40], [item_sk#45], Inner, (rk#44 >= rk#49), BuildRight +(46) ReusedExchange [Reuses operator id: 41] +Output [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] -(66) CometProject -Input [9]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, rk#44, item_sk#45, web_sales#47, store_sales#48, rk#49] -Arguments: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48], [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48] +(47) CometSort +Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +Arguments: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38], [ss_item_sk#38 ASC NULLS FIRST, d_date#36 ASC NULLS FIRST] -(67) CometHashAggregate -Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_sales#47, store_sales#48] -Keys [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] -Functions [2]: [partial_max(web_sales#47), partial_max(store_sales#48)] - -(68) CometHashAggregate -Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, max#50, max#51] -Keys [4]: [item_sk#40, d_date#41, web_sales#42, store_sales#43] -Functions [2]: [max(web_sales#47), max(store_sales#48)] - -(69) CometFilter -Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] -Condition : ((isnotnull(web_cumulative#52) AND isnotnull(store_cumulative#53)) AND (web_cumulative#52 > store_cumulative#53)) - -(70) CometTakeOrderedAndProject -Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#40 ASC NULLS FIRST,d_date#41 ASC NULLS FIRST], output=[item_sk#40,d_date#41,web_sales#42,store_sales#43,web_cumulative#52,store_cumulative#53]), [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53], 100, 0, [item_sk#40 ASC NULLS FIRST, d_date#41 ASC NULLS FIRST], [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] - -(71) CometColumnarToRow [codegen id : 9] -Input [6]: [item_sk#40, d_date#41, web_sales#42, store_sales#43, web_cumulative#52, store_cumulative#53] +(48) CometColumnarToRow [codegen id : 11] +Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] + +(49) Window +Input [4]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38] +Arguments: [row_number() windowspecdefinition(ss_item_sk#38, d_date#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#39], [ss_item_sk#38], [d_date#36 ASC NULLS FIRST] + +(50) Project [codegen id : 12] +Output [3]: [item_sk#35, sumss#37, rk#39] +Input [5]: [item_sk#35, d_date#36, sumss#37, ss_item_sk#38, rk#39] + +(51) BroadcastExchange +Input [3]: [item_sk#35, sumss#37, rk#39] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +(52) BroadcastHashJoin [codegen id : 13] +Left keys [1]: [item_sk#32] +Right keys [1]: [item_sk#35] +Join type: Inner +Join condition: (rk#34 >= rk#39) + +(53) Project [codegen id : 13] +Output [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] +Input [7]: [item_sk#32, d_date#29, sumss#33, rk#34, item_sk#35, sumss#37, rk#39] + +(54) HashAggregate [codegen id : 13] +Input [4]: [item_sk#32, d_date#29, sumss#33, sumss#37] +Keys [3]: [item_sk#32, d_date#29, sumss#33] +Functions [1]: [partial_sum(sumss#37)] +Aggregate Attributes [2]: [sum#40, isEmpty#41] +Results [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] + +(55) CometColumnarExchange +Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +Arguments: hashpartitioning(item_sk#32, d_date#29, sumss#33, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(56) CometColumnarToRow [codegen id : 14] +Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] + +(57) HashAggregate [codegen id : 14] +Input [5]: [item_sk#32, d_date#29, sumss#33, sum#42, isEmpty#43] +Keys [3]: [item_sk#32, d_date#29, sumss#33] +Functions [1]: [sum(sumss#37)] +Aggregate Attributes [1]: [sum(sumss#37)#44] +Results [3]: [item_sk#32, d_date#29, sum(sumss#37)#44 AS cume_sales#45] + +(58) CometColumnarExchange +Input [3]: [item_sk#32, d_date#29, cume_sales#45] +Arguments: hashpartitioning(item_sk#32, d_date#29, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(59) CometSort +Input [3]: [item_sk#32, d_date#29, cume_sales#45] +Arguments: [item_sk#32, d_date#29, cume_sales#45], [item_sk#32 ASC NULLS FIRST, d_date#29 ASC NULLS FIRST] + +(60) CometSortMergeJoin +Left output [3]: [item_sk#10, d_date#6, cume_sales#23] +Right output [3]: [item_sk#32, d_date#29, cume_sales#45] +Arguments: [item_sk#10, d_date#6], [item_sk#32, d_date#29], FullOuter + +(61) CometFilter +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END) + +(62) CometProject +Input [6]: [item_sk#10, d_date#6, cume_sales#23, item_sk#32, d_date#29, cume_sales#45] +Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [CASE WHEN isnotnull(item_sk#10) THEN item_sk#10 ELSE item_sk#32 END AS item_sk#46, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#29 END AS d_date#47, cume_sales#23 AS web_sales#48, cume_sales#45 AS store_sales#49] + +(63) CometExchange +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: hashpartitioning(item_sk#46, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(64) CometSort +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [item_sk#46, d_date#47, web_sales#48, store_sales#49], [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST] + +(65) CometColumnarToRow [codegen id : 15] +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] + +(66) Window +Input [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Arguments: [row_number() windowspecdefinition(item_sk#46, d_date#47 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#50], [item_sk#46], [d_date#47 ASC NULLS FIRST] + +(67) ReusedExchange [Reuses operator id: 63] +Output [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] + +(68) CometSort +Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +Arguments: [item_sk#51, d_date#52, web_sales#53, store_sales#54], [item_sk#51 ASC NULLS FIRST, d_date#52 ASC NULLS FIRST] + +(69) CometColumnarToRow [codegen id : 30] +Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] + +(70) Window +Input [4]: [item_sk#51, d_date#52, web_sales#53, store_sales#54] +Arguments: [row_number() windowspecdefinition(item_sk#51, d_date#52 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#55], [item_sk#51], [d_date#52 ASC NULLS FIRST] + +(71) Project [codegen id : 31] +Output [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] +Input [5]: [item_sk#51, d_date#52, web_sales#53, store_sales#54, rk#55] + +(72) BroadcastExchange +Input [4]: [item_sk#51, web_sales#53, store_sales#54, rk#55] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] + +(73) BroadcastHashJoin [codegen id : 32] +Left keys [1]: [item_sk#46] +Right keys [1]: [item_sk#51] +Join type: Inner +Join condition: (rk#50 >= rk#55) + +(74) Project [codegen id : 32] +Output [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] +Input [9]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, rk#50, item_sk#51, web_sales#53, store_sales#54, rk#55] + +(75) HashAggregate [codegen id : 32] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_sales#53, store_sales#54] +Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Functions [2]: [partial_max(web_sales#53), partial_max(store_sales#54)] +Aggregate Attributes [2]: [max#56, max#57] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] + +(76) HashAggregate [codegen id : 32] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max#58, max#59] +Keys [4]: [item_sk#46, d_date#47, web_sales#48, store_sales#49] +Functions [2]: [max(web_sales#53), max(store_sales#54)] +Aggregate Attributes [2]: [max(web_sales#53)#60, max(store_sales#54)#61] +Results [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, max(web_sales#53)#60 AS web_cumulative#62, max(store_sales#54)#61 AS store_cumulative#63] + +(77) Filter [codegen id : 32] +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +Condition : ((isnotnull(web_cumulative#62) AND isnotnull(store_cumulative#63)) AND (web_cumulative#62 > store_cumulative#63)) + +(78) TakeOrderedAndProject +Input [6]: [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] +Arguments: 100, [item_sk#46 ASC NULLS FIRST, d_date#47 ASC NULLS FIRST], [item_sk#46, d_date#47, web_sales#48, store_sales#49, web_cumulative#62, store_cumulative#63] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (76) -+- * CometColumnarToRow (75) - +- CometProject (74) - +- CometFilter (73) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(73) CometFilter +(80) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(74) CometProject +(81) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(75) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(76) BroadcastExchange +(83) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 29 Hosting Expression = ss_sold_date_sk#23 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 32 Hosting Expression = ss_sold_date_sk#26 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt index 6e3fd36930..f7546ea154 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/extended.txt @@ -1,211 +1,224 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometWindowExec - : : : +- CometSort - : : : +- CometColumnarExchange - : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : : +- CometColumnarToRow - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- HashAggregate + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- Window [COMET: Window expressions are not supported] + : : : +- CometColumnarToRow + : : : +- CometSort + : : : +- CometColumnarExchange + : : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : : +- CometColumnarToRow + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ws_item_sk#1 AS item_sk#2, d_date#3, MakeDecimal(sum(UnscaledValue(ws_sales_price#4))#5,17,2) AS sumws#6, ws_item_sk#1)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(ss_item_sk#7 AS item_sk#8, d_date#9, MakeDecimal(sum(UnscaledValue(ss_sales_price#10))#11,17,2) AS sumss#12, ss_item_sk#7)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 176 out of 196 eligible operators (89%). Final plan contains 13 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 130 out of 196 eligible operators (66%). Final plan contains 26 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt index 51d1e38ec6..c88883dbad 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q51a/simplified.txt @@ -1,86 +1,122 @@ -WholeStageCodegen (9) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometHashAggregate [max,max] [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative,max(web_sales),max(store_sales)] - CometHashAggregate [web_sales,store_sales] [item_sk,d_date,web_sales,store_sales,max,max] - CometProject [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - CometBroadcastHashJoin [item_sk,d_date,web_sales,store_sales,rk,item_sk,web_sales,store_sales,rk] - CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #2 - CometHashAggregate [sumws,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumws)] - CometExchange [item_sk,d_date,sumws] #3 - CometHashAggregate [sumws] [item_sk,d_date,sumws,sum,isEmpty] - CometProject [item_sk,d_date,sumws,sumws] - CometBroadcastHashJoin [item_sk,d_date,sumws,rk,item_sk,sumws,rk] - CometProject [item_sk,d_date,sumws,rk] - CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometColumnarExchange [ws_item_sk] #4 - WholeStageCodegen (1) - HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] - CometColumnarToRow +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (32) + Filter [web_cumulative,store_cumulative] + HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (15) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (7) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (6) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] InputAdapter - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [item_sk,sumws,rk] #8 - CometProject [item_sk,sumws,rk] - CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #9 - CometHashAggregate [sumss,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumss)] - CometExchange [item_sk,d_date,sumss] #10 - CometHashAggregate [sumss] [item_sk,d_date,sumss,sum,isEmpty] - CometProject [item_sk,d_date,sumss,sumss] - CometBroadcastHashJoin [item_sk,d_date,sumss,rk,item_sk,sumss,rk] - CometProject [item_sk,d_date,sumss,rk] - CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometColumnarExchange [ss_item_sk] #11 - WholeStageCodegen (3) - HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] - CometColumnarToRow + Window [ws_item_sk,d_date] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometColumnarExchange [ws_item_sk] #4 + WholeStageCodegen (1) + HashAggregate [ws_item_sk,d_date,sum] [sum(UnscaledValue(ws_sales_price)),item_sk,sumws,sum] + CometColumnarToRow + InputAdapter + CometExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (5) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #9 + WholeStageCodegen (14) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + WholeStageCodegen (13) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] InputAdapter - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 - CometBroadcastExchange [item_sk,sumss,rk] #13 - CometProject [item_sk,sumss,rk] - CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - CometBroadcastExchange [item_sk,web_sales,store_sales,rk] #14 - CometProject [item_sk,web_sales,store_sales,rk] - CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] - CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + Window [ss_item_sk,d_date] + WholeStageCodegen (9) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometColumnarExchange [ss_item_sk] #11 + WholeStageCodegen (8) + HashAggregate [ss_item_sk,d_date,sum] [sum(UnscaledValue(ss_sales_price)),item_sk,sumss,sum] + CometColumnarToRow + InputAdapter + CometExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (12) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (11) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (31) + Project [item_sk,web_sales,store_sales,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (30) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt index 0181f2dc7f..e5b34056ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/explain.txt @@ -1,48 +1,54 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (29) - : : +- CometFilter (28) - : : +- CometWindowExec (27) - : : +- CometFilter (26) - : : +- CometWindowExec (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometWindowExec (35) - : +- CometSort (34) - : +- CometColumnarExchange (33) - : +- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- ReusedExchange (30) - +- ReusedExchange (40) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -160,112 +166,136 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 2] Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometFilter +(26) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(27) Filter [codegen id : 3] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) CometWindowExec +(28) Window Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(28) CometFilter +(29) Filter [codegen id : 10] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(29) CometProject +(30) Project [codegen id : 10] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(30) ReusedExchange [Reuses operator id: 20] +(31) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(31) CometColumnarToRow [codegen id : 2] +(32) CometColumnarToRow [codegen id : 4] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(32) HashAggregate [codegen id : 2] +(33) HashAggregate [codegen id : 4] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] -(33) CometColumnarExchange +(34) CometColumnarExchange Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(34) CometSort +(35) CometSort Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(35) CometWindowExec +(36) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] + +(37) Window Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30], [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(36) CometProject +(38) Project [codegen id : 6] +Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] -Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30], [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(37) CometBroadcastExchange +(39) BroadcastExchange Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(38) CometBroadcastHashJoin -Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Right output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)], Inner, BuildRight +(40) BroadcastHashJoin [codegen id : 10] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] +Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)] +Join type: Inner +Join condition: None -(39) CometProject +(41) Project [codegen id : 10] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -(40) ReusedExchange [Reuses operator id: 37] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +(42) ReusedExchange [Reuses operator id: 34] +Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] + +(43) CometSort +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(44) CometColumnarToRow [codegen id : 8] +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] + +(45) Window +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(46) Project [codegen id : 9] +Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] -(41) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -Right output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#31, i_brand#32, cc_name#33, (rn#35 - 1)], Inner, BuildRight +(47) BroadcastExchange +Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(42) CometProject -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -Arguments: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#36, sum_sales#34 AS nsum#37] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] +Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 1)] +Join type: Inner +Join condition: None -(43) CometTakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#36,nsum#37]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +(49) Project [codegen id : 10] +Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -(44) CometColumnarToRow [codegen id : 4] -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +(50) TakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(52) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(47) CometColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(48) BroadcastExchange +(54) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt index 8d50c97859..1c4e1f9890 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/extended.txt @@ -1,78 +1,80 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] @@ -103,4 +105,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt index 8f7680b9f5..29a73f88ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57.native_iceberg_compat/simplified.txt @@ -1,57 +1,77 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,cc_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (2) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] + WholeStageCodegen (10) + Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (3) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (2) CometColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt index 0181f2dc7f..e5b34056ab 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/explain.txt @@ -1,48 +1,54 @@ == Physical Plan == -* CometColumnarToRow (44) -+- CometTakeOrderedAndProject (43) - +- CometProject (42) - +- CometBroadcastHashJoin (41) - :- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (29) - : : +- CometFilter (28) - : : +- CometWindowExec (27) - : : +- CometFilter (26) - : : +- CometWindowExec (25) - : : +- CometSort (24) - : : +- CometColumnarExchange (23) - : : +- * HashAggregate (22) - : : +- * CometColumnarToRow (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometWindowExec (35) - : +- CometSort (34) - : +- CometColumnarExchange (33) - : +- * HashAggregate (32) - : +- * CometColumnarToRow (31) - : +- ReusedExchange (30) - +- ReusedExchange (40) +TakeOrderedAndProject (50) ++- * Project (49) + +- * BroadcastHashJoin Inner BuildRight (48) + :- * Project (41) + : +- * BroadcastHashJoin Inner BuildRight (40) + : :- * Project (30) + : : +- * Filter (29) + : : +- Window (28) + : : +- * Filter (27) + : : +- Window (26) + : : +- * CometColumnarToRow (25) + : : +- CometSort (24) + : : +- CometColumnarExchange (23) + : : +- * HashAggregate (22) + : : +- * CometColumnarToRow (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (39) + : +- * Project (38) + : +- Window (37) + : +- * CometColumnarToRow (36) + : +- CometSort (35) + : +- CometColumnarExchange (34) + : +- * HashAggregate (33) + : +- * CometColumnarToRow (32) + : +- ReusedExchange (31) + +- BroadcastExchange (47) + +- * Project (46) + +- Window (45) + +- * CometColumnarToRow (44) + +- CometSort (43) + +- ReusedExchange (42) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -160,112 +166,136 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometWindowExec +(25) CometColumnarToRow [codegen id : 2] Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(26) CometFilter +(26) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(27) Filter [codegen id : 3] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(27) CometWindowExec +(28) Window Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(28) CometFilter +(29) Filter [codegen id : 10] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(29) CometProject +(30) Project [codegen id : 10] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(30) ReusedExchange [Reuses operator id: 20] +(31) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(31) CometColumnarToRow [codegen id : 2] +(32) CometColumnarToRow [codegen id : 4] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] -(32) HashAggregate [codegen id : 2] +(33) HashAggregate [codegen id : 4] Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum#27] Keys [5]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26] Functions [1]: [sum(UnscaledValue(cs_sales_price#28))] Aggregate Attributes [1]: [sum(UnscaledValue(cs_sales_price#28))#17] Results [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, MakeDecimal(sum(UnscaledValue(cs_sales_price#28))#17,17,2) AS sum_sales#29] -(33) CometColumnarExchange +(34) CometColumnarExchange Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: hashpartitioning(i_category#22, i_brand#23, cc_name#24, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(34) CometSort +(35) CometSort Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29], [i_category#22 ASC NULLS FIRST, i_brand#23 ASC NULLS FIRST, cc_name#24 ASC NULLS FIRST, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(35) CometWindowExec +(36) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] + +(37) Window Input [6]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29] -Arguments: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30], [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] +Arguments: [rank(d_year#25, d_moy#26) windowspecdefinition(i_category#22, i_brand#23, cc_name#24, d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#30], [i_category#22, i_brand#23, cc_name#24], [d_year#25 ASC NULLS FIRST, d_moy#26 ASC NULLS FIRST] -(36) CometProject +(38) Project [codegen id : 6] +Output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] Input [7]: [i_category#22, i_brand#23, cc_name#24, d_year#25, d_moy#26, sum_sales#29, rn#30] -Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30], [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -(37) CometBroadcastExchange +(39) BroadcastExchange Input [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(38) CometBroadcastHashJoin -Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Right output [5]: [i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)], Inner, BuildRight +(40) BroadcastHashJoin [codegen id : 10] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] +Right keys [4]: [i_category#22, i_brand#23, cc_name#24, (rn#30 + 1)] +Join type: Inner +Join condition: None -(39) CometProject +(41) Project [codegen id : 10] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#22, i_brand#23, cc_name#24, sum_sales#29, rn#30] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -(40) ReusedExchange [Reuses operator id: 37] -Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] +(42) ReusedExchange [Reuses operator id: 34] +Output [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] + +(43) CometSort +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +Arguments: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36], [i_category#31 ASC NULLS FIRST, i_brand#32 ASC NULLS FIRST, cc_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(44) CometColumnarToRow [codegen id : 8] +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] + +(45) Window +Input [6]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#31, i_brand#32, cc_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#37], [i_category#31, i_brand#32, cc_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(46) Project [codegen id : 9] +Output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +Input [7]: [i_category#31, i_brand#32, cc_name#33, d_year#34, d_moy#35, sum_sales#36, rn#37] -(41) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29] -Right output [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#20], [i_category#31, i_brand#32, cc_name#33, (rn#35 - 1)], Inner, BuildRight +(47) BroadcastExchange +Input [5]: [i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(42) CometProject -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#34, rn#35] -Arguments: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#36, sum_sales#34 AS nsum#37] +(48) BroadcastHashJoin [codegen id : 10] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#20] +Right keys [4]: [i_category#31, i_brand#32, cc_name#33, (rn#37 - 1)] +Join type: Inner +Join condition: None -(43) CometTakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#36,nsum#37]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +(49) Project [codegen id : 10] +Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#29 AS psum#38, sum_sales#36 AS nsum#39] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#29, i_category#31, i_brand#32, cc_name#33, sum_sales#36, rn#37] -(44) CometColumnarToRow [codegen id : 4] -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#36, nsum#37] +(50) TakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (48) -+- * CometColumnarToRow (47) - +- CometFilter (46) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (45) +BroadcastExchange (54) ++- * CometColumnarToRow (53) + +- CometFilter (52) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (51) -(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(51) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(46) CometFilter +(52) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(47) CometColumnarToRow [codegen id : 1] +(53) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(48) BroadcastExchange +(54) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt index 8d50c97859..1c4e1f9890 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/extended.txt @@ -1,78 +1,80 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometColumnarExchange - : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] - : : +- CometColumnarToRow - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometColumnarExchange - : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] - : +- CometColumnarToRow - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#1, i_brand#2, cc_name#3, d_year#4, d_moy#5, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS sum_sales#8, MakeDecimal(sum(UnscaledValue(cs_sales_price#6))#7,17,2) AS _w0#9)] + : : +- CometColumnarToRow + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] + : +- CometColumnarToRow + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometColumnarExchange +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_category#10, i_brand#11, cc_name#12, d_year#13, d_moy#14, MakeDecimal(sum(UnscaledValue(cs_sales_price#15))#7,17,2) AS sum_sales#16)] @@ -103,4 +105,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 88 out of 97 eligible operators (90%). Final plan contains 7 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 97 eligible operators (74%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt index 8f7680b9f5..29a73f88ac 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q57/simplified.txt @@ -1,57 +1,77 @@ -WholeStageCodegen (4) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometColumnarExchange [i_category,i_brand,cc_name] #1 - WholeStageCodegen (1) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,cc_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometColumnarExchange [i_category,i_brand,cc_name] #8 - WholeStageCodegen (2) - HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] + WholeStageCodegen (10) + Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (3) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (2) CometColumnarToRow InputAdapter - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometColumnarExchange [i_category,i_brand,cc_name] #1 + WholeStageCodegen (1) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (6) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometColumnarExchange [i_category,i_brand,cc_name] #8 + WholeStageCodegen (4) + HashAggregate [i_category,i_brand,cc_name,d_year,d_moy,sum] [sum(UnscaledValue(cs_sales_price)),sum_sales,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (9) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (8) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt index ac091ccfdd..ae9541ff99 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (66) -+- CometTakeOrderedAndProject (65) - +- CometProject (64) - +- CometWindowExec (63) +TakeOrderedAndProject (66) ++- * Project (65) + +- Window (64) + +- * CometColumnarToRow (63) +- CometSort (62) +- CometExchange (61) +- CometHashAggregate (60) @@ -358,20 +358,20 @@ Arguments: hashpartitioning(lochierarchy#31, _w0#63, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63], [lochierarchy#31 ASC NULLS FIRST, _w0#63 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST] -(63) CometWindowExec +(63) CometColumnarToRow [codegen id : 19] Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] -Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64], [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#63, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#64], [lochierarchy#31, _w0#63], [total_sum#26 DESC NULLS LAST] -(64) CometProject -Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64] -Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +(64) Window +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] +Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#63, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#64], [lochierarchy#31, _w0#63], [total_sum#26 DESC NULLS LAST] -(65) CometTakeOrderedAndProject -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#31 DESC NULLS LAST,CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST,rank_within_parent#64 ASC NULLS FIRST], output=[total_sum#26,s_state#27,s_county#28,lochierarchy#31,rank_within_parent#64]), [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64], 100, 0, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#64 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +(65) Project [codegen id : 20] +Output [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64] -(66) CometColumnarToRow [codegen id : 19] +(66) TakeOrderedAndProject Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +Arguments: 100, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#64 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt index d73b57ca68..fda8aba347 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -173,4 +173,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 99 out of 156 eligible operators (63%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 96 out of 156 eligible operators (61%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt index 3f4387c7ca..21c6dd4eb1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a.native_iceberg_compat/simplified.txt @@ -1,99 +1,101 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,rank_within_parent] - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (20) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + WholeStageCodegen (18) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + CometColumnarToRow + InputAdapter + CometExchange [s_state] #8 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #9 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (11) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #10 + WholeStageCodegen (10) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #11 + WholeStageCodegen (16) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt index ac091ccfdd..ae9541ff99 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (66) -+- CometTakeOrderedAndProject (65) - +- CometProject (64) - +- CometWindowExec (63) +TakeOrderedAndProject (66) ++- * Project (65) + +- Window (64) + +- * CometColumnarToRow (63) +- CometSort (62) +- CometExchange (61) +- CometHashAggregate (60) @@ -358,20 +358,20 @@ Arguments: hashpartitioning(lochierarchy#31, _w0#63, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63], [lochierarchy#31 ASC NULLS FIRST, _w0#63 ASC NULLS FIRST, total_sum#26 DESC NULLS LAST] -(63) CometWindowExec +(63) CometColumnarToRow [codegen id : 19] Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] -Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64], [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#63, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#64], [lochierarchy#31, _w0#63], [total_sum#26 DESC NULLS LAST] -(64) CometProject -Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64] -Arguments: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +(64) Window +Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63] +Arguments: [rank(total_sum#26) windowspecdefinition(lochierarchy#31, _w0#63, total_sum#26 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#64], [lochierarchy#31, _w0#63], [total_sum#26 DESC NULLS LAST] -(65) CometTakeOrderedAndProject -Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#31 DESC NULLS LAST,CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST,rank_within_parent#64 ASC NULLS FIRST], output=[total_sum#26,s_state#27,s_county#28,lochierarchy#31,rank_within_parent#64]), [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64], 100, 0, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#64 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +(65) Project [codegen id : 20] +Output [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +Input [6]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, _w0#63, rank_within_parent#64] -(66) CometColumnarToRow [codegen id : 19] +(66) TakeOrderedAndProject Input [5]: [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] +Arguments: 100, [lochierarchy#31 DESC NULLS LAST, CASE WHEN (lochierarchy#31 = 0) THEN s_state#27 END ASC NULLS FIRST, rank_within_parent#64 ASC NULLS FIRST], [total_sum#26, s_state#27, s_county#28, lochierarchy#31, rank_within_parent#64] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt index d73b57ca68..fda8aba347 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -173,4 +173,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 99 out of 156 eligible operators (63%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 96 out of 156 eligible operators (61%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt index 3f4387c7ca..21c6dd4eb1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q70a/simplified.txt @@ -1,99 +1,101 @@ -WholeStageCodegen (19) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,rank_within_parent] - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - WholeStageCodegen (18) - HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state,s_county] #3 - WholeStageCodegen (4) - HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] - Project [ss_net_profit,s_county,s_state] - BroadcastHashJoin [ss_store_sk,s_store_sk] - CometColumnarToRow - InputAdapter - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - InputAdapter - BroadcastExchange #6 - WholeStageCodegen (3) - Project [s_store_sk,s_county,s_state] - BroadcastHashJoin [s_state,s_state] - CometColumnarToRow - InputAdapter - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - InputAdapter - BroadcastExchange #7 - WholeStageCodegen (2) - Project [s_state] - Filter [ranking] - InputAdapter - Window [_w0,s_state] - WindowGroupLimit [s_state,_w0] - WholeStageCodegen (1) - Sort [s_state,_w0] - HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - WholeStageCodegen (11) - HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [s_state] #10 - WholeStageCodegen (10) - HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #11 - WholeStageCodegen (16) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [s_state,s_county,sum] #3 +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (20) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + WholeStageCodegen (18) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WindowGroupLimit [s_state,_w0] + WholeStageCodegen (1) + Sort [s_state,_w0] + HashAggregate [sum] [sum(UnscaledValue(ss_net_profit)),_w0,s_state,sum] + CometColumnarToRow + InputAdapter + CometExchange [s_state] #8 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #9 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (11) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #10 + WholeStageCodegen (10) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #11 + WholeStageCodegen (16) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt index b8e5e4175c..07622631b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometWindowExec (39) +TakeOrderedAndProject (42) ++- * Project (41) + +- Window (40) + +- * CometColumnarToRow (39) +- CometSort (38) +- CometExchange (37) +- CometHashAggregate (36) @@ -224,20 +224,20 @@ Arguments: hashpartitioning(lochierarchy#19, _w0#51, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51], [lochierarchy#19 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] -(39) CometWindowExec +(39) CometColumnarToRow [codegen id : 7] Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52], [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] -(40) CometProject -Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52] -Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +(40) Window +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] +Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] -(41) CometTakeOrderedAndProject -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#19 DESC NULLS LAST,CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST,rank_within_parent#52 ASC NULLS FIRST], output=[total_sum#14,i_category#15,i_class#16,lochierarchy#19,rank_within_parent#52]), [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52], 100, 0, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +(41) Project [codegen id : 8] +Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52] -(42) CometColumnarToRow [codegen id : 7] +(42) TakeOrderedAndProject Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt index 9dea0e7de6..2beaad68b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -89,4 +89,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 66 out of 81 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 81 eligible operators (77%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt index d1903ae6b4..d9db895fba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a.native_iceberg_compat/simplified.txt @@ -1,64 +1,66 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,rank_within_parent] - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (6) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (2) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (5) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (4) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (8) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + WholeStageCodegen (6) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (1) + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class] #3 + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (3) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #7 + WholeStageCodegen (2) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 + WholeStageCodegen (5) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #8 + WholeStageCodegen (4) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt index b8e5e4175c..07622631b8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometWindowExec (39) +TakeOrderedAndProject (42) ++- * Project (41) + +- Window (40) + +- * CometColumnarToRow (39) +- CometSort (38) +- CometExchange (37) +- CometHashAggregate (36) @@ -224,20 +224,20 @@ Arguments: hashpartitioning(lochierarchy#19, _w0#51, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51], [lochierarchy#19 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#14 DESC NULLS LAST] -(39) CometWindowExec +(39) CometColumnarToRow [codegen id : 7] Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] -Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52], [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] -(40) CometProject -Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52] -Arguments: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +(40) Window +Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51] +Arguments: [rank(total_sum#14) windowspecdefinition(lochierarchy#19, _w0#51, total_sum#14 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#19, _w0#51], [total_sum#14 DESC NULLS LAST] -(41) CometTakeOrderedAndProject -Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#19 DESC NULLS LAST,CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST,rank_within_parent#52 ASC NULLS FIRST], output=[total_sum#14,i_category#15,i_class#16,lochierarchy#19,rank_within_parent#52]), [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52], 100, 0, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +(41) Project [codegen id : 8] +Output [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +Input [6]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, _w0#51, rank_within_parent#52] -(42) CometColumnarToRow [codegen id : 7] +(42) TakeOrderedAndProject Input [5]: [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] +Arguments: 100, [lochierarchy#19 DESC NULLS LAST, CASE WHEN (lochierarchy#19 = 0) THEN i_category#15 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#14, i_category#15, i_class#16, lochierarchy#19, rank_within_parent#52] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt index 9dea0e7de6..2beaad68b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -89,4 +89,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 66 out of 81 eligible operators (81%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 63 out of 81 eligible operators (77%). Final plan contains 9 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt index d1903ae6b4..d9db895fba 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q86a/simplified.txt @@ -1,64 +1,66 @@ -WholeStageCodegen (7) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,rank_within_parent] - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - WholeStageCodegen (6) - HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - WholeStageCodegen (3) - HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [i_category] #7 - WholeStageCodegen (2) - HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 - WholeStageCodegen (5) - HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #8 - WholeStageCodegen (4) - HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] - HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] - CometColumnarToRow - InputAdapter - ReusedExchange [i_category,i_class,sum] #3 +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (8) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometColumnarExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + WholeStageCodegen (6) + HashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + InputAdapter + Union + WholeStageCodegen (1) + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_category,i_class] #3 + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + WholeStageCodegen (3) + HashAggregate [i_category,sum,isEmpty] [sum(total_sum),total_sum,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [i_category] #7 + WholeStageCodegen (2) + HashAggregate [i_category,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 + WholeStageCodegen (5) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #8 + WholeStageCodegen (4) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [i_category,i_class,sum] [sum(UnscaledValue(ws_net_paid)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt index 92a6fc5a00..4b24124687 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/explain.txt @@ -1,29 +1,30 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometExchange (23) - +- CometProject (22) - +- CometWindowExec (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (26) ++- CometSort (25) + +- CometColumnarExchange (24) + +- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -121,54 +122,57 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject +(22) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(23) Project [codegen id : 3] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -(23) CometExchange +(24) CometColumnarExchange Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometSort +(25) CometSort Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 2] +(26) CometColumnarToRow [codegen id : 4] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +BroadcastExchange (31) ++- * CometColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(28) CometProject +(29) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt index 5ee2c01b35..7517dafcbe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/extended.txt @@ -1,33 +1,34 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS _w0#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS _w0#9)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt index 8067c884b0..1a79190465 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98.native_iceberg_compat/simplified.txt @@ -1,37 +1,42 @@ -WholeStageCodegen (2) +WholeStageCodegen (4) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (3) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt index 92a6fc5a00..4b24124687 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/explain.txt @@ -1,29 +1,30 @@ == Physical Plan == -* CometColumnarToRow (25) -+- CometSort (24) - +- CometExchange (23) - +- CometProject (22) - +- CometWindowExec (21) - +- CometSort (20) - +- CometColumnarExchange (19) - +- * HashAggregate (18) - +- * CometColumnarToRow (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (26) ++- CometSort (25) + +- CometColumnarExchange (24) + +- * Project (23) + +- Window (22) + +- * CometColumnarToRow (21) + +- CometSort (20) + +- CometColumnarExchange (19) + +- * HashAggregate (18) + +- * CometColumnarToRow (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -121,54 +122,57 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometColumnarSh Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19], [i_class#12 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 2] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20], [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] -(22) CometProject +(22) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19] +Arguments: [sum(_w0#19) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#20], [i_class#12] + +(23) Project [codegen id : 3] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, _w0#19, _we0#20] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, ((_w0#19 * 100) / _we0#20) AS revenueratio#21] -(23) CometExchange +(24) CometColumnarExchange Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(24) CometSort +(25) CometSort Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#21 ASC NULLS FIRST] -(25) CometColumnarToRow [codegen id : 2] +(26) CometColumnarToRow [codegen id : 4] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#18, revenueratio#21] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (30) -+- * CometColumnarToRow (29) - +- CometProject (28) - +- CometFilter (27) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) +BroadcastExchange (31) ++- * CometColumnarToRow (30) + +- CometProject (29) + +- CometFilter (28) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (27) -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(28) CometProject +(29) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(29) CometColumnarToRow [codegen id : 1] +(30) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(30) BroadcastExchange +(31) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt index 5ee2c01b35..7517dafcbe 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/extended.txt @@ -1,33 +1,34 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometColumnarExchange - +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS _w0#9)] - +- CometColumnarToRow - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometColumnarExchange + +- HashAggregate [COMET: makedecimal is not supported, Unsupported result expressions found in: List(i_item_id#1, i_item_desc#2, i_category#3, i_class#4, i_current_price#5, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS itemrevenue#8, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#6))#7,17,2) AS _w0#9)] + +- CometColumnarToRow + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 28 eligible operators (89%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 23 out of 28 eligible operators (82%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt index 8067c884b0..1a79190465 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark4_0/q98/simplified.txt @@ -1,37 +1,42 @@ -WholeStageCodegen (2) +WholeStageCodegen (4) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometColumnarExchange [i_class] #2 - WholeStageCodegen (1) - HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] - CometColumnarToRow - InputAdapter - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (3) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometColumnarExchange [i_class] #2 + WholeStageCodegen (1) + HashAggregate [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] [sum(UnscaledValue(ss_ext_sales_price)),itemrevenue,_w0,sum] + CometColumnarToRow + InputAdapter + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt index db7ab3e60c..410aecff1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometProject (21) - +- CometWindowExec (20) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +(21) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(22) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -(23) CometColumnarToRow [codegen id : 1] +(23) TakeOrderedAndProject Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt index 7b9553ba1c..9ddd3aeb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt index 63d522180c..58ac81ba0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.native_iceberg_compat/simplified.txt @@ -1,33 +1,35 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt index db7ab3e60c..410aecff1b 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometProject (21) - +- CometWindowExec (20) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +(21) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(22) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -(23) CometColumnarToRow [codegen id : 1] +(23) TakeOrderedAndProject Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt index 7b9553ba1c..9ddd3aeb38 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt index 63d522180c..58ac81ba0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/simplified.txt @@ -1,33 +1,35 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ws_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [ws_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ws_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ws_ext_sales_price,ws_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_ext_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt index e4b07ef2a8..322bbe63b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometProject (21) - +- CometWindowExec (20) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +(21) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(22) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -(23) CometColumnarToRow [codegen id : 1] +(23) TakeOrderedAndProject Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt index 3bd3bd46f8..400633ed9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt index 173aea3be4..cf18e68a3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.native_iceberg_compat/simplified.txt @@ -1,33 +1,35 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt index e4b07ef2a8..322bbe63b3 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (23) -+- CometTakeOrderedAndProject (22) - +- CometProject (21) - +- CometWindowExec (20) +TakeOrderedAndProject (23) ++- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) +- CometSort (19) +- CometExchange (18) +- CometHashAggregate (17) @@ -114,20 +114,20 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject -Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +(21) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(22) CometTakeOrderedAndProject -Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#13 ASC NULLS FIRST,i_class#12 ASC NULLS FIRST,i_item_id#11 ASC NULLS FIRST,i_item_desc#7 ASC NULLS FIRST,revenueratio#20 ASC NULLS FIRST], output=[i_item_id#11,i_item_desc#7,i_category#13,i_class#12,i_current_price#8,itemrevenue#17,revenueratio#20]), [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], 100, 0, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] +Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -(23) CometColumnarToRow [codegen id : 1] +(23) TakeOrderedAndProject Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] +Arguments: 100, [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt index 3bd3bd46f8..400633ed9f 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -28,4 +28,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 25 out of 27 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 22 out of 27 eligible operators (81%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt index 173aea3be4..cf18e68a3d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/simplified.txt @@ -1,33 +1,35 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #1 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 - CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] +TakeOrderedAndProject [i_category,i_class,i_item_id,i_item_desc,revenueratio,i_current_price,itemrevenue] + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #1 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(cs_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #2 + CometHashAggregate [cs_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [cs_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [cs_ext_sales_price,cs_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_ext_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #4 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt index c0a8c11952..0f966424f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (43) -+- CometTakeOrderedAndProject (42) - +- CometProject (41) - +- CometWindowExec (40) +TakeOrderedAndProject (43) ++- * Project (42) + +- Window (41) + +- * CometColumnarToRow (40) +- CometSort (39) +- CometExchange (38) +- CometHashAggregate (37) @@ -225,20 +225,20 @@ Arguments: hashpartitioning(lochierarchy#41, _w0#53, 5), ENSURE_REQUIREMENTS, Co Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53], [lochierarchy#41 ASC NULLS FIRST, _w0#53 ASC NULLS FIRST, gross_margin#36 ASC NULLS FIRST] -(40) CometWindowExec +(40) CometColumnarToRow [codegen id : 1] Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] -Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54], [rank(gross_margin#36) windowspecdefinition(lochierarchy#41, _w0#53, gross_margin#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#54], [lochierarchy#41, _w0#53], [gross_margin#36 ASC NULLS FIRST] -(41) CometProject -Input [6]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54] -Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +(41) Window +Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] +Arguments: [rank(gross_margin#36) windowspecdefinition(lochierarchy#41, _w0#53, gross_margin#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#54], [lochierarchy#41, _w0#53], [gross_margin#36 ASC NULLS FIRST] -(42) CometTakeOrderedAndProject -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#41 DESC NULLS LAST,CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST,rank_within_parent#54 ASC NULLS FIRST], output=[gross_margin#36,i_category#37,i_class#38,lochierarchy#41,rank_within_parent#54]), [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54], 100, 0, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST, rank_within_parent#54 ASC NULLS FIRST], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +(42) Project [codegen id : 2] +Output [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +Input [6]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54] -(43) CometColumnarToRow [codegen id : 1] +(43) TakeOrderedAndProject Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +Arguments: 100, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST, rank_within_parent#54 ASC NULLS FIRST], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt index cd5c2362b7..dee6669b47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt index 70d1ad26f6..e91b278c14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.native_iceberg_compat/simplified.txt @@ -1,53 +1,55 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,rank_within_parent] - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (2) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [gross_margin,lochierarchy,_w0] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange [i_category] #8 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange #9 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt index c0a8c11952..0f966424f8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (43) -+- CometTakeOrderedAndProject (42) - +- CometProject (41) - +- CometWindowExec (40) +TakeOrderedAndProject (43) ++- * Project (42) + +- Window (41) + +- * CometColumnarToRow (40) +- CometSort (39) +- CometExchange (38) +- CometHashAggregate (37) @@ -225,20 +225,20 @@ Arguments: hashpartitioning(lochierarchy#41, _w0#53, 5), ENSURE_REQUIREMENTS, Co Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53], [lochierarchy#41 ASC NULLS FIRST, _w0#53 ASC NULLS FIRST, gross_margin#36 ASC NULLS FIRST] -(40) CometWindowExec +(40) CometColumnarToRow [codegen id : 1] Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] -Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54], [rank(gross_margin#36) windowspecdefinition(lochierarchy#41, _w0#53, gross_margin#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#54], [lochierarchy#41, _w0#53], [gross_margin#36 ASC NULLS FIRST] -(41) CometProject -Input [6]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54] -Arguments: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +(41) Window +Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53] +Arguments: [rank(gross_margin#36) windowspecdefinition(lochierarchy#41, _w0#53, gross_margin#36 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#54], [lochierarchy#41, _w0#53], [gross_margin#36 ASC NULLS FIRST] -(42) CometTakeOrderedAndProject -Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#41 DESC NULLS LAST,CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST,rank_within_parent#54 ASC NULLS FIRST], output=[gross_margin#36,i_category#37,i_class#38,lochierarchy#41,rank_within_parent#54]), [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54], 100, 0, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST, rank_within_parent#54 ASC NULLS FIRST], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +(42) Project [codegen id : 2] +Output [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +Input [6]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, _w0#53, rank_within_parent#54] -(43) CometColumnarToRow [codegen id : 1] +(43) TakeOrderedAndProject Input [5]: [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] +Arguments: 100, [lochierarchy#41 DESC NULLS LAST, CASE WHEN (lochierarchy#41 = 0) THEN i_category#37 END ASC NULLS FIRST, rank_within_parent#54 ASC NULLS FIRST], [gross_margin#36, i_category#37, i_class#38, lochierarchy#41, rank_within_parent#54] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt index cd5c2362b7..dee6669b47 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -102,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 93 out of 99 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 90 out of 99 eligible operators (90%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt index 70d1ad26f6..e91b278c14 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/simplified.txt @@ -1,53 +1,55 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [gross_margin,i_category,i_class,lochierarchy,_w0,rank_within_parent] - CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] - CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 - CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] - CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] - CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] - CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] - CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometBroadcastExchange [s_store_sk] #7 - CometProject [s_store_sk] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange [i_category] #8 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 - CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] - CometExchange #9 - CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] - CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] - ReusedExchange [i_category,i_class,sum,sum] #3 +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,gross_margin,i_class] + WholeStageCodegen (2) + Project [gross_margin,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [gross_margin,lochierarchy,_w0] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [gross_margin,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [t_category] [gross_margin,i_category,i_class,lochierarchy,_w0,t_class] + CometExchange [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] #2 + CometHashAggregate [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometUnion [gross_margin,i_category,i_class,t_category,t_class,lochierarchy] + CometHashAggregate [sum,sum] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,i_class,sum,sum] + CometProject [ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category,s_store_sk] + CometProject [ss_store_sk,ss_ext_sales_price,ss_net_profit,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,i_item_sk,i_class,i_category] + CometProject [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometBroadcastExchange [s_store_sk] #7 + CometProject [s_store_sk] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange [i_category] #8 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [i_category,sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,i_category,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 + CometHashAggregate [sum,isEmpty,sum,isEmpty] [gross_margin,i_category,i_class,t_category,t_class,lochierarchy,sum(ss_net_profit),sum(ss_ext_sales_price)] + CometExchange #9 + CometHashAggregate [ss_net_profit,ss_ext_sales_price] [sum,isEmpty,sum,isEmpty] + CometHashAggregate [i_category,i_class,sum,sum] [ss_net_profit,ss_ext_sales_price,sum(UnscaledValue(ss_net_profit)),sum(UnscaledValue(ss_ext_sales_price))] + ReusedExchange [i_category,i_class,sum,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt index 53bc60ca3a..7b1f860f32 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/explain.txt @@ -1,46 +1,52 @@ == Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (28) - : : +- CometFilter (27) - : : +- CometWindowExec (26) - : : +- CometFilter (25) - : : +- CometWindowExec (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometWindowExec (33) - : +- CometSort (32) - : +- CometExchange (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - +- ReusedExchange (38) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -153,107 +159,131 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometWindowExec +(24) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometFilter +(25) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(26) Filter [codegen id : 2] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(26) CometWindowExec +(27) Window Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(27) CometFilter +(28) Filter [codegen id : 7] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(28) CometProject +(29) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(29) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -(30) CometHashAggregate +(31) CometHashAggregate Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] -(31) CometExchange +(32) CometExchange Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) CometSort +(33) CometSort Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(33) CometWindowExec +(34) CometColumnarToRow [codegen id : 3] +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] + +(35) Window Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28], [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(34) CometProject +(36) Project [codegen id : 4] +Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] -Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28], [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -(35) CometBroadcastExchange +(37) BroadcastExchange Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(36) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Right output [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)], Inner, BuildRight +(38) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] +Join type: Inner +Join condition: None -(37) CometProject +(39) Project [codegen id : 7] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -(38) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +(40) ReusedExchange [Reuses operator id: 32] +Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] + +(41) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(42) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] + +(43) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(44) Project [codegen id : 6] +Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] +Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] -(39) CometBroadcastHashJoin -Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Right output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#37 - 1)], Inner, BuildRight +(45) BroadcastExchange +Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(40) CometProject -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -Arguments: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#38, sum_sales#36 AS nsum#39] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] +Join type: Inner +Join condition: None -(41) CometTakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#38,nsum#39]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(47) Project [codegen id : 7] +Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -(42) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(48) TakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(45) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(46) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt index 8c361a8340..11fbf5bd11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/extended.txt @@ -1,76 +1,78 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -100,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt index c8c2cc4f95..a62c33ecc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.native_iceberg_compat/simplified.txt @@ -1,51 +1,71 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] + WholeStageCodegen (7) + Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (2) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt index 53bc60ca3a..7b1f860f32 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt @@ -1,46 +1,52 @@ == Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (28) - : : +- CometFilter (27) - : : +- CometWindowExec (26) - : : +- CometFilter (25) - : : +- CometWindowExec (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometWindowExec (33) - : +- CometSort (32) - : +- CometExchange (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - +- ReusedExchange (38) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -153,107 +159,131 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#15, s_company_ Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#15 ASC NULLS FIRST, s_company_name#16 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometWindowExec +(24) CometColumnarToRow [codegen id : 1] Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometFilter +(25) Window +Input [8]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(26) Filter [codegen id : 2] Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(26) CometWindowExec +(27) Window Input [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21], [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] +Arguments: [avg(_w0#19) windowspecdefinition(i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#21], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12] -(27) CometFilter +(28) Filter [codegen id : 7] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND CASE WHEN (avg_monthly_sales#21 > 0.000000) THEN ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000) END) -(28) CometProject +(29) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] Input [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -(29) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] -(30) CometHashAggregate +(31) CometHashAggregate Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum#26] Keys [6]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25] Functions [1]: [sum(UnscaledValue(ss_sales_price#27))] -(31) CometExchange +(32) CometExchange Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: hashpartitioning(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) CometSort +(33) CometSort Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#22 ASC NULLS FIRST, s_company_name#23 ASC NULLS FIRST, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(33) CometWindowExec +(34) CometColumnarToRow [codegen id : 3] +Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] + +(35) Window Input [7]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18] -Arguments: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28], [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] +Arguments: [rank(d_year#24, d_moy#25) windowspecdefinition(i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#28], [i_category#5, i_brand#4, s_store_name#22, s_company_name#23], [d_year#24 ASC NULLS FIRST, d_moy#25 ASC NULLS FIRST] -(34) CometProject +(36) Project [codegen id : 4] +Output [6]: [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] Input [8]: [i_category#5, i_brand#4, s_store_name#22, s_company_name#23, d_year#24, d_moy#25, sum_sales#18, rn#28] -Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28], [i_category#5 AS i_category#29, i_brand#4 AS i_brand#30, s_store_name#22, s_company_name#23, sum_sales#18 AS sum_sales#31, rn#28] -(35) CometBroadcastExchange +(37) BroadcastExchange Input [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] + 1)),false), [plan_id=4] -(36) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20] -Right output [6]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)], Inner, BuildRight +(38) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#29, i_brand#30, s_store_name#22, s_company_name#23, (rn#28 + 1)] +Join type: Inner +Join condition: None -(37) CometProject +(39) Project [codegen id : 7] +Output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] Input [15]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, i_category#29, i_brand#30, s_store_name#22, s_company_name#23, sum_sales#31, rn#28] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31], [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -(38) ReusedExchange [Reuses operator id: 35] -Output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] +(40) ReusedExchange [Reuses operator id: 32] +Output [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] + +(41) CometSort +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, s_store_name#32 ASC NULLS FIRST, s_company_name#33 ASC NULLS FIRST, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(42) CometColumnarToRow [codegen id : 5] +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] + +(43) Window +Input [7]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18] +Arguments: [rank(d_year#34, d_moy#35) windowspecdefinition(i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#36], [i_category#5, i_brand#4, s_store_name#32, s_company_name#33], [d_year#34 ASC NULLS FIRST, d_moy#35 ASC NULLS FIRST] + +(44) Project [codegen id : 6] +Output [6]: [i_category#5 AS i_category#37, i_brand#4 AS i_brand#38, s_store_name#32, s_company_name#33, sum_sales#18 AS sum_sales#39, rn#36] +Input [8]: [i_category#5, i_brand#4, s_store_name#32, s_company_name#33, d_year#34, d_moy#35, sum_sales#18, rn#36] -(39) CometBroadcastHashJoin -Left output [10]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31] -Right output [6]: [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -Arguments: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20], [i_category#32, i_brand#33, s_store_name#34, s_company_name#35, (rn#37 - 1)], Inner, BuildRight +(45) BroadcastExchange +Input [6]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], input[3, string, true], (input[5, int, false] - 1)),false), [plan_id=5] -(40) CometProject -Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#32, i_brand#33, s_store_name#34, s_company_name#35, sum_sales#36, rn#37] -Arguments: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#38, sum_sales#36 AS nsum#39] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [5]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, rn#20] +Right keys [5]: [i_category#37, i_brand#38, s_store_name#32, s_company_name#33, (rn#36 - 1)] +Join type: Inner +Join condition: None -(41) CometTakeOrderedAndProject -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST,d_moy#13 ASC NULLS FIRST], output=[i_category#5,d_year#12,d_moy#13,avg_monthly_sales#21,sum_sales#18,psum#38,nsum#39]), [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39], 100, 0, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(47) Project [codegen id : 7] +Output [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, sum_sales#31 AS psum#40, sum_sales#39 AS nsum#41] +Input [16]: [i_category#5, i_brand#4, s_store_name#15, s_company_name#16, d_year#12, d_moy#13, sum_sales#18, avg_monthly_sales#21, rn#20, sum_sales#31, i_category#37, i_brand#38, s_store_name#32, s_company_name#33, sum_sales#39, rn#36] -(42) CometColumnarToRow [codegen id : 1] -Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#38, nsum#39] +(48) TakeOrderedAndProject +Input [7]: [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] +Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST], [i_category#5, d_year#12, d_moy#13, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(45) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(46) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt index 8c361a8340..11fbf5bd11 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/extended.txt @@ -1,76 +1,78 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -100,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store -Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt index c8c2cc4f95..a62c33ecc1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/simplified.txt @@ -1,51 +1,71 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometProject [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] - CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 - CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 - CometFilter [s_store_sk,s_store_name,s_company_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] - CometBroadcastExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 - CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] #7 +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_moy,i_category,d_year,psum,nsum] + WholeStageCodegen (7) + Project [i_category,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,s_store_name,s_company_name,rn,i_category,i_brand,s_store_name,s_company_name,rn] + Project [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,s_store_name,s_company_name,d_year] + WholeStageCodegen (2) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #1 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(ss_sales_price))] + CometExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy] #2 + CometHashAggregate [ss_sales_price] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,ss_sales_price,d_year,d_moy,s_store_name,s_company_name] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy,s_store_sk,s_store_name,s_company_name] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [s_store_sk,s_store_name,s_company_name] #6 + CometFilter [s_store_sk,s_store_name,s_company_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_name,s_company_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,s_store_name,s_company_name] #8 + CometHashAggregate [sum] [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales,sum(UnscaledValue(ss_sales_price))] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,s_store_name,s_company_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,s_store_name,s_company_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,s_store_name,s_company_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt index 2c993b48e0..74702d596d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/explain.txt @@ -1,79 +1,82 @@ == Physical Plan == -* CometColumnarToRow (75) -+- CometTakeOrderedAndProject (74) - +- CometHashAggregate (73) - +- CometExchange (72) - +- CometHashAggregate (71) - +- CometUnion (70) - :- CometProject (25) - : +- CometFilter (24) - : +- CometWindowExec (23) - : +- CometSort (22) - : +- CometWindowExec (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- CometProject (47) - : +- CometFilter (46) - : +- CometWindowExec (45) - : +- CometSort (44) - : +- CometWindowExec (43) - : +- CometSort (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometBroadcastExchange (29) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (35) - +- CometProject (69) - +- CometFilter (68) - +- CometWindowExec (67) - +- CometSort (66) - +- CometWindowExec (65) - +- CometSort (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometProject (56) - : +- CometBroadcastHashJoin (55) - : :- CometBroadcastExchange (51) - : : +- CometProject (50) - : : +- CometFilter (49) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) - +- ReusedExchange (57) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -170,27 +173,30 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) CometSort +(22) Window +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] + +(23) Sort [codegen id : 2] Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 -(23) CometWindowExec +(24) Window Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(24) CometFilter +(25) Filter [codegen id : 3] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(25) CometProject +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -198,97 +204,100 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(28) CometProject +(29) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(29) CometBroadcastExchange +(30) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(32) CometProject +(33) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(33) CometBroadcastHashJoin +(34) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(34) CometProject +(35) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(35) ReusedExchange [Reuses operator id: 13] +(36) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(36) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(37) CometProject +(38) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(38) CometHashAggregate +(39) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(39) CometExchange +(40) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) CometHashAggregate +(41) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(41) CometExchange +(42) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(42) CometSort +(43) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(43) CometWindowExec +(44) CometColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] + +(45) Window Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(44) CometSort +(46) Sort [codegen id : 5] Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 -(45) CometWindowExec +(47) Window Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(46) CometFilter +(48) Filter [codegen id : 6] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(47) CometProject +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -296,156 +305,158 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(49) CometFilter +(51) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(50) CometProject +(52) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(51) CometBroadcastExchange +(53) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(54) CometProject +(56) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(56) CometProject +(58) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(57) ReusedExchange [Reuses operator id: 13] +(59) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(59) CometProject +(61) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(60) CometHashAggregate +(62) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(61) CometExchange +(63) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(62) CometHashAggregate +(64) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(65) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(64) CometSort +(66) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(65) CometWindowExec +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(66) CometSort +(68) Window +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] + +(69) Sort [codegen id : 8] Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 -(67) CometWindowExec +(70) Window Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(68) CometFilter +(71) Filter [codegen id : 9] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(69) CometProject +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(70) CometUnion -Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(73) Union -(71) CometHashAggregate +(74) HashAggregate [codegen id : 10] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(72) CometExchange +(75) CometColumnarExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometHashAggregate +(76) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(74) CometTakeOrderedAndProject +(77) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter +(80) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(78) CometProject +(81) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(79) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(80) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt index 6a8de7fe28..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/extended.txt @@ -1,91 +1,94 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- Filter + +- Window + +- Sort + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt index 4f128e83de..ca80833ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.native_iceberg_compat/simplified.txt @@ -1,87 +1,110 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometUnion [channel,item,return_ratio,return_rank,currency_rank] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt index 2c993b48e0..74702d596d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt @@ -1,79 +1,82 @@ == Physical Plan == -* CometColumnarToRow (75) -+- CometTakeOrderedAndProject (74) - +- CometHashAggregate (73) - +- CometExchange (72) - +- CometHashAggregate (71) - +- CometUnion (70) - :- CometProject (25) - : +- CometFilter (24) - : +- CometWindowExec (23) - : +- CometSort (22) - : +- CometWindowExec (21) - : +- CometSort (20) - : +- CometExchange (19) - : +- CometHashAggregate (18) - : +- CometExchange (17) - : +- CometHashAggregate (16) - : +- CometProject (15) - : +- CometBroadcastHashJoin (14) - : :- CometProject (9) - : : +- CometBroadcastHashJoin (8) - : : :- CometBroadcastExchange (4) - : : : +- CometProject (3) - : : : +- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : +- CometProject (7) - : : +- CometFilter (6) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) - : +- CometBroadcastExchange (13) - : +- CometProject (12) - : +- CometFilter (11) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) - :- CometProject (47) - : +- CometFilter (46) - : +- CometWindowExec (45) - : +- CometSort (44) - : +- CometWindowExec (43) - : +- CometSort (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometExchange (39) - : +- CometHashAggregate (38) - : +- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (34) - : : +- CometBroadcastHashJoin (33) - : : :- CometBroadcastExchange (29) - : : : +- CometProject (28) - : : : +- CometFilter (27) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (26) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (30) - : +- ReusedExchange (35) - +- CometProject (69) - +- CometFilter (68) - +- CometWindowExec (67) - +- CometSort (66) - +- CometWindowExec (65) - +- CometSort (64) - +- CometExchange (63) - +- CometHashAggregate (62) - +- CometExchange (61) - +- CometHashAggregate (60) - +- CometProject (59) - +- CometBroadcastHashJoin (58) - :- CometProject (56) - : +- CometBroadcastHashJoin (55) - : :- CometBroadcastExchange (51) - : : +- CometProject (50) - : : +- CometFilter (49) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (48) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (52) - +- ReusedExchange (57) +* CometColumnarToRow (78) ++- CometTakeOrderedAndProject (77) + +- CometHashAggregate (76) + +- CometColumnarExchange (75) + +- * HashAggregate (74) + +- Union (73) + :- * Project (26) + : +- * Filter (25) + : +- Window (24) + : +- * Sort (23) + : +- Window (22) + : +- * CometColumnarToRow (21) + : +- CometSort (20) + : +- CometExchange (19) + : +- CometHashAggregate (18) + : +- CometExchange (17) + : +- CometHashAggregate (16) + : +- CometProject (15) + : +- CometBroadcastHashJoin (14) + : :- CometProject (9) + : : +- CometBroadcastHashJoin (8) + : : :- CometBroadcastExchange (4) + : : : +- CometProject (3) + : : : +- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : +- CometProject (7) + : : +- CometFilter (6) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (5) + : +- CometBroadcastExchange (13) + : +- CometProject (12) + : +- CometFilter (11) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (10) + :- * Project (49) + : +- * Filter (48) + : +- Window (47) + : +- * Sort (46) + : +- Window (45) + : +- * CometColumnarToRow (44) + : +- CometSort (43) + : +- CometExchange (42) + : +- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometBroadcastExchange (30) + : : : +- CometProject (29) + : : : +- CometFilter (28) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (27) + : : +- CometProject (33) + : : +- CometFilter (32) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (31) + : +- ReusedExchange (36) + +- * Project (72) + +- * Filter (71) + +- Window (70) + +- * Sort (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- CometExchange (65) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (58) + : +- CometBroadcastHashJoin (57) + : :- CometBroadcastExchange (53) + : : +- CometProject (52) + : : +- CometFilter (51) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (50) + : +- CometProject (56) + : +- CometFilter (55) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (54) + +- ReusedExchange (59) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -170,27 +173,30 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] Input [3]: [item#22, return_ratio#23, currency_ratio#24] Arguments: [item#22, return_ratio#23, currency_ratio#24], [return_ratio#23 ASC NULLS FIRST] -(21) CometWindowExec +(21) CometColumnarToRow [codegen id : 1] Input [3]: [item#22, return_ratio#23, currency_ratio#24] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] -(22) CometSort +(22) Window +Input [3]: [item#22, return_ratio#23, currency_ratio#24] +Arguments: [rank(return_ratio#23) windowspecdefinition(return_ratio#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#25], [return_ratio#23 ASC NULLS FIRST] + +(23) Sort [codegen id : 2] Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [currency_ratio#24 ASC NULLS FIRST], false, 0 -(23) CometWindowExec +(24) Window Input [4]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25] -Arguments: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26], [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#24) windowspecdefinition(currency_ratio#24 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#26], [currency_ratio#24 ASC NULLS FIRST] -(24) CometFilter +(25) Filter [codegen id : 3] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] Condition : ((return_rank#25 <= 10) OR (currency_rank#26 <= 10)) -(25) CometProject +(26) Project [codegen id : 3] +Output [5]: [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Input [5]: [item#22, return_ratio#23, currency_ratio#24, return_rank#25, currency_rank#26] -Arguments: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], [web AS channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales Output [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Batched: true Location: InMemoryFileIndex [] @@ -198,97 +204,100 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#33), dynamicpruningexpression(cs_so PushedFilters: [IsNotNull(cs_net_profit), IsNotNull(cs_net_paid), IsNotNull(cs_quantity), GreaterThan(cs_net_profit,1.00), GreaterThan(cs_net_paid,0.00), GreaterThan(cs_quantity,0), IsNotNull(cs_order_number), IsNotNull(cs_item_sk)] ReadSchema: struct -(27) CometFilter +(28) CometFilter Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Condition : (((((((isnotnull(cs_net_profit#32) AND isnotnull(cs_net_paid#31)) AND isnotnull(cs_quantity#30)) AND (cs_net_profit#32 > 1.00)) AND (cs_net_paid#31 > 0.00)) AND (cs_quantity#30 > 0)) AND isnotnull(cs_order_number#29)) AND isnotnull(cs_item_sk#28)) -(28) CometProject +(29) CometProject Input [6]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_net_profit#32, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33], [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(29) CometBroadcastExchange +(30) CometBroadcastExchange Input [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Arguments: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] -(30) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns Output [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_return_amount), GreaterThan(cr_return_amount,10000.00), IsNotNull(cr_order_number), IsNotNull(cr_item_sk)] ReadSchema: struct -(31) CometFilter +(32) CometFilter Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Condition : (((isnotnull(cr_return_amount#38) AND (cr_return_amount#38 > 10000.00)) AND isnotnull(cr_order_number#36)) AND isnotnull(cr_item_sk#35)) -(32) CometProject +(33) CometProject Input [5]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38, cr_returned_date_sk#39] Arguments: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38], [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] -(33) CometBroadcastHashJoin +(34) CometBroadcastHashJoin Left output [5]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33] Right output [4]: [cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_order_number#29, cs_item_sk#28], [cr_order_number#36, cr_item_sk#35], Inner, BuildLeft -(34) CometProject +(35) CometProject Input [9]: [cs_item_sk#28, cs_order_number#29, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_item_sk#35, cr_order_number#36, cr_return_quantity#37, cr_return_amount#38] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] -(35) ReusedExchange [Reuses operator id: 13] +(36) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#40] -(36) CometBroadcastHashJoin +(37) CometBroadcastHashJoin Left output [6]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38] Right output [1]: [d_date_sk#40] Arguments: [cs_sold_date_sk#33], [d_date_sk#40], Inner, BuildRight -(37) CometProject +(38) CometProject Input [7]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cs_sold_date_sk#33, cr_return_quantity#37, cr_return_amount#38, d_date_sk#40] Arguments: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38], [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] -(38) CometHashAggregate +(39) CometHashAggregate Input [5]: [cs_item_sk#28, cs_quantity#30, cs_net_paid#31, cr_return_quantity#37, cr_return_amount#38] Keys [1]: [cs_item_sk#28] Functions [4]: [partial_sum(coalesce(cr_return_quantity#37, 0)), partial_sum(coalesce(cs_quantity#30, 0)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(39) CometExchange +(40) CometExchange Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Arguments: hashpartitioning(cs_item_sk#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(40) CometHashAggregate +(41) CometHashAggregate Input [7]: [cs_item_sk#28, sum#41, sum#42, sum#43, isEmpty#44, sum#45, isEmpty#46] Keys [1]: [cs_item_sk#28] Functions [4]: [sum(coalesce(cr_return_quantity#37, 0)), sum(coalesce(cs_quantity#30, 0)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#31 as decimal(12,2)), 0.00))] -(41) CometExchange +(42) CometExchange Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(42) CometSort +(43) CometSort Input [3]: [item#47, return_ratio#48, currency_ratio#49] Arguments: [item#47, return_ratio#48, currency_ratio#49], [return_ratio#48 ASC NULLS FIRST] -(43) CometWindowExec +(44) CometColumnarToRow [codegen id : 4] +Input [3]: [item#47, return_ratio#48, currency_ratio#49] + +(45) Window Input [3]: [item#47, return_ratio#48, currency_ratio#49] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] +Arguments: [rank(return_ratio#48) windowspecdefinition(return_ratio#48 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#50], [return_ratio#48 ASC NULLS FIRST] -(44) CometSort +(46) Sort [codegen id : 5] Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [currency_ratio#49 ASC NULLS FIRST], false, 0 -(45) CometWindowExec +(47) Window Input [4]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50] -Arguments: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51], [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#49) windowspecdefinition(currency_ratio#49 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#51], [currency_ratio#49 ASC NULLS FIRST] -(46) CometFilter +(48) Filter [codegen id : 6] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] Condition : ((return_rank#50 <= 10) OR (currency_rank#51 <= 10)) -(47) CometProject +(49) Project [codegen id : 6] +Output [5]: [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] Input [5]: [item#47, return_ratio#48, currency_ratio#49, return_rank#50, currency_rank#51] -Arguments: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51], [catalog AS channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -(48) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(50) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] @@ -296,156 +305,158 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_net_profit), IsNotNull(ss_net_paid), IsNotNull(ss_quantity), GreaterThan(ss_net_profit,1.00), GreaterThan(ss_net_paid,0.00), GreaterThan(ss_quantity,0), IsNotNull(ss_ticket_number), IsNotNull(ss_item_sk)] ReadSchema: struct -(49) CometFilter +(51) CometFilter Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Condition : (((((((isnotnull(ss_net_profit#57) AND isnotnull(ss_net_paid#56)) AND isnotnull(ss_quantity#55)) AND (ss_net_profit#57 > 1.00)) AND (ss_net_paid#56 > 0.00)) AND (ss_quantity#55 > 0)) AND isnotnull(ss_ticket_number#54)) AND isnotnull(ss_item_sk#53)) -(50) CometProject +(52) CometProject Input [6]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_net_profit#57, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58], [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(51) CometBroadcastExchange +(53) CometBroadcastExchange Input [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Arguments: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] -(52) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns +(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns Output [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/store_returns] PushedFilters: [IsNotNull(sr_return_amt), GreaterThan(sr_return_amt,10000.00), IsNotNull(sr_ticket_number), IsNotNull(sr_item_sk)] ReadSchema: struct -(53) CometFilter +(55) CometFilter Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Condition : (((isnotnull(sr_return_amt#63) AND (sr_return_amt#63 > 10000.00)) AND isnotnull(sr_ticket_number#61)) AND isnotnull(sr_item_sk#60)) -(54) CometProject +(56) CometProject Input [5]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63, sr_returned_date_sk#64] Arguments: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63], [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] -(55) CometBroadcastHashJoin +(57) CometBroadcastHashJoin Left output [5]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58] Right output [4]: [sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_ticket_number#54, ss_item_sk#53], [sr_ticket_number#61, sr_item_sk#60], Inner, BuildLeft -(56) CometProject +(58) CometProject Input [9]: [ss_item_sk#53, ss_ticket_number#54, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_item_sk#60, sr_ticket_number#61, sr_return_quantity#62, sr_return_amt#63] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] -(57) ReusedExchange [Reuses operator id: 13] +(59) ReusedExchange [Reuses operator id: 13] Output [1]: [d_date_sk#65] -(58) CometBroadcastHashJoin +(60) CometBroadcastHashJoin Left output [6]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63] Right output [1]: [d_date_sk#65] Arguments: [ss_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(59) CometProject +(61) CometProject Input [7]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, ss_sold_date_sk#58, sr_return_quantity#62, sr_return_amt#63, d_date_sk#65] Arguments: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63], [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] -(60) CometHashAggregate +(62) CometHashAggregate Input [5]: [ss_item_sk#53, ss_quantity#55, ss_net_paid#56, sr_return_quantity#62, sr_return_amt#63] Keys [1]: [ss_item_sk#53] Functions [4]: [partial_sum(coalesce(sr_return_quantity#62, 0)), partial_sum(coalesce(ss_quantity#55, 0)), partial_sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), partial_sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(61) CometExchange +(63) CometExchange Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Arguments: hashpartitioning(ss_item_sk#53, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(62) CometHashAggregate +(64) CometHashAggregate Input [7]: [ss_item_sk#53, sum#66, sum#67, sum#68, isEmpty#69, sum#70, isEmpty#71] Keys [1]: [ss_item_sk#53] Functions [4]: [sum(coalesce(sr_return_quantity#62, 0)), sum(coalesce(ss_quantity#55, 0)), sum(coalesce(cast(sr_return_amt#63 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#56 as decimal(12,2)), 0.00))] -(63) CometExchange +(65) CometExchange Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(64) CometSort +(66) CometSort Input [3]: [item#72, return_ratio#73, currency_ratio#74] Arguments: [item#72, return_ratio#73, currency_ratio#74], [return_ratio#73 ASC NULLS FIRST] -(65) CometWindowExec +(67) CometColumnarToRow [codegen id : 7] Input [3]: [item#72, return_ratio#73, currency_ratio#74] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] -(66) CometSort +(68) Window +Input [3]: [item#72, return_ratio#73, currency_ratio#74] +Arguments: [rank(return_ratio#73) windowspecdefinition(return_ratio#73 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS return_rank#75], [return_ratio#73 ASC NULLS FIRST] + +(69) Sort [codegen id : 8] Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [currency_ratio#74 ASC NULLS FIRST], false, 0 -(67) CometWindowExec +(70) Window Input [4]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75] -Arguments: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76], [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] +Arguments: [rank(currency_ratio#74) windowspecdefinition(currency_ratio#74 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS currency_rank#76], [currency_ratio#74 ASC NULLS FIRST] -(68) CometFilter +(71) Filter [codegen id : 9] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] Condition : ((return_rank#75 <= 10) OR (currency_rank#76 <= 10)) -(69) CometProject +(72) Project [codegen id : 9] +Output [5]: [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] Input [5]: [item#72, return_ratio#73, currency_ratio#74, return_rank#75, currency_rank#76] -Arguments: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76], [store AS channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] -(70) CometUnion -Child 0 Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Child 1 Input [5]: [channel#52, item#47, return_ratio#48, return_rank#50, currency_rank#51] -Child 2 Input [5]: [channel#77, item#72, return_ratio#73, return_rank#75, currency_rank#76] +(73) Union -(71) CometHashAggregate +(74) HashAggregate [codegen id : 10] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] +Aggregate Attributes: [] +Results [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(72) CometExchange +(75) CometColumnarExchange Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +Arguments: hashpartitioning(channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(73) CometHashAggregate +(76) CometHashAggregate Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Keys [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Functions: [] -(74) CometTakeOrderedAndProject +(77) CometTakeOrderedAndProject Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#27 ASC NULLS FIRST,return_rank#25 ASC NULLS FIRST,currency_rank#26 ASC NULLS FIRST,item#22 ASC NULLS FIRST], output=[channel#27,item#22,return_ratio#23,return_rank#25,currency_rank#26]), [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26], 100, 0, [channel#27 ASC NULLS FIRST, return_rank#25 ASC NULLS FIRST, currency_rank#26 ASC NULLS FIRST, item#22 ASC NULLS FIRST], [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] -(75) CometColumnarToRow [codegen id : 1] +(78) CometColumnarToRow [codegen id : 11] Input [5]: [channel#27, item#22, return_ratio#23, return_rank#25, currency_rank#26] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#6 IN dynamicpruning#7 -BroadcastExchange (80) -+- * CometColumnarToRow (79) - +- CometProject (78) - +- CometFilter (77) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (76) +BroadcastExchange (83) ++- * CometColumnarToRow (82) + +- CometProject (81) + +- CometFilter (80) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (79) -(76) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(79) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#13, d_year#14, d_moy#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2001), EqualTo(d_moy,12), IsNotNull(d_date_sk)] ReadSchema: struct -(77) CometFilter +(80) CometFilter Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Condition : ((((isnotnull(d_year#14) AND isnotnull(d_moy#15)) AND (d_year#14 = 2001)) AND (d_moy#15 = 12)) AND isnotnull(d_date_sk#13)) -(78) CometProject +(81) CometProject Input [3]: [d_date_sk#13, d_year#14, d_moy#15] Arguments: [d_date_sk#13], [d_date_sk#13] -(79) CometColumnarToRow [codegen id : 1] +(82) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#13] -(80) BroadcastExchange +(83) BroadcastExchange Input [1]: [d_date_sk#13] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] -Subquery:2 Hosting operator id = 26 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 +Subquery:2 Hosting operator id = 27 Hosting Expression = cs_sold_date_sk#33 IN dynamicpruning#7 -Subquery:3 Hosting operator id = 48 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 +Subquery:3 Hosting operator id = 50 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#7 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt index 6a8de7fe28..734b0ce7bb 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/extended.txt @@ -1,91 +1,94 @@ CometColumnarToRow +- CometTakeOrderedAndProject +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- ReusedSubquery - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- HashAggregate + +- Union + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- Project + : +- Filter + : +- Window + : +- Sort + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- ReusedSubquery + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- Project + +- Filter + +- Window + +- Sort + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 83 out of 87 eligible operators (95%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 66 out of 87 eligible operators (75%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt index 4f128e83de..ca80833ee5 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/simplified.txt @@ -1,87 +1,110 @@ -WholeStageCodegen (1) +WholeStageCodegen (11) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,item,return_ratio,return_rank,currency_rank] CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometExchange [channel,item,return_ratio,return_rank,currency_rank] #1 - CometHashAggregate [channel,item,return_ratio,return_rank,currency_rank] - CometUnion [channel,item,return_ratio,return_rank,currency_rank] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #2 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] - CometExchange [ws_item_sk] #3 - CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] - CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] - CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 - CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] - CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] - CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #7 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] - CometExchange [cs_item_sk] #8 - CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] - CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] - CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 - CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] - CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] - CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometProject [channel,item,return_ratio,return_rank,currency_rank] - CometFilter [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank,currency_rank] - CometSort [item,return_ratio,currency_ratio,return_rank] - CometWindowExec [item,return_ratio,currency_ratio,return_rank] - CometSort [item,return_ratio,currency_ratio] - CometExchange #10 - CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] - CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] - CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 - CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] - CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] - ReusedExchange [d_date_sk] #6 + CometColumnarExchange [channel,item,return_ratio,return_rank,currency_rank] #1 + WholeStageCodegen (10) + HashAggregate [channel,item,return_ratio,return_rank,currency_rank] + InputAdapter + Union + WholeStageCodegen (3) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (2) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #2 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ws_item_sk,sum(coalesce(wr_return_quantity, 0)),sum(coalesce(ws_quantity, 0)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ws_net_paid as decimal(12,2)), 0.00))] + CometExchange [ws_item_sk] #3 + CometHashAggregate [wr_return_quantity,ws_quantity,wr_return_amt,ws_net_paid] [ws_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt,d_date_sk] + CometProject [ws_item_sk,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_return_quantity,wr_return_amt] + CometBroadcastHashJoin [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometBroadcastExchange [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] #4 + CometProject [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_sold_date_sk] + CometFilter [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_net_paid,ws_net_profit,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometProject [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt] + CometFilter [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + WholeStageCodegen (6) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (5) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (4) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #7 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,cs_item_sk,sum(coalesce(cr_return_quantity, 0)),sum(coalesce(cs_quantity, 0)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(coalesce(cast(cs_net_paid as decimal(12,2)), 0.00))] + CometExchange [cs_item_sk] #8 + CometHashAggregate [cr_return_quantity,cs_quantity,cr_return_amount,cs_net_paid] [cs_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount,d_date_sk] + CometProject [cs_item_sk,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_return_quantity,cr_return_amount] + CometBroadcastHashJoin [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometBroadcastExchange [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] #9 + CometProject [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_sold_date_sk] + CometFilter [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_net_paid,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount] + CometFilter [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + WholeStageCodegen (9) + Project [item,return_ratio,return_rank,currency_rank] + Filter [return_rank,currency_rank] + InputAdapter + Window [currency_ratio] + WholeStageCodegen (8) + Sort [currency_ratio] + InputAdapter + Window [return_ratio] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item,return_ratio,currency_ratio] + CometExchange #10 + CometHashAggregate [sum,sum,sum,isEmpty,sum,isEmpty] [item,return_ratio,currency_ratio,ss_item_sk,sum(coalesce(sr_return_quantity, 0)),sum(coalesce(ss_quantity, 0)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(coalesce(cast(ss_net_paid as decimal(12,2)), 0.00))] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sr_return_quantity,ss_quantity,sr_return_amt,ss_net_paid] [ss_item_sk,sum,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt,d_date_sk] + CometProject [ss_item_sk,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_return_quantity,sr_return_amt] + CometBroadcastHashJoin [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometBroadcastExchange [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] #12 + CometProject [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_sold_date_sk] + CometFilter [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_net_paid,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk] + ReusedExchange [d_date_sk] #6 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt index ce43cefb57..f8a95a6db8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/explain.txt @@ -1,73 +1,80 @@ == Physical Plan == -* CometColumnarToRow (69) -+- CometTakeOrderedAndProject (68) - +- CometFilter (67) - +- CometHashAggregate (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometWindowExec (57) - : +- CometSort (56) - : +- CometExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometSortMergeJoin (52) - : :- CometSort (27) - : : +- CometExchange (26) - : : +- CometHashAggregate (25) - : : +- CometExchange (24) - : : +- CometHashAggregate (23) - : : +- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometProject (15) - : : : +- CometWindowExec (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometWindowExec (18) - : : +- CometSort (17) - : : +- ReusedExchange (16) - : +- CometSort (51) - : +- CometExchange (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (39) - : : +- CometWindowExec (38) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometHashAggregate (35) - : : +- CometExchange (34) - : : +- CometHashAggregate (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometFilter (29) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) - : : +- ReusedExchange (30) - : +- CometBroadcastExchange (44) - : +- CometProject (43) - : +- CometWindowExec (42) - : +- CometSort (41) - : +- ReusedExchange (40) - +- CometBroadcastExchange (62) - +- CometProject (61) - +- CometWindowExec (60) - +- CometSort (59) - +- ReusedExchange (58) +TakeOrderedAndProject (76) ++- * Filter (75) + +- * HashAggregate (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- Window (64) + : +- * CometColumnarToRow (63) + : +- CometSort (62) + : +- CometExchange (61) + : +- CometProject (60) + : +- CometFilter (59) + : +- CometSortMergeJoin (58) + : :- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- * HashAggregate (28) + : : +- * CometColumnarToRow (27) + : : +- CometColumnarExchange (26) + : : +- * HashAggregate (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (16) + : : : +- Window (15) + : : : +- * CometColumnarToRow (14) + : : : +- CometSort (13) + : : : +- CometExchange (12) + : : : +- CometHashAggregate (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (22) + : : +- * Project (21) + : : +- Window (20) + : : +- * CometColumnarToRow (19) + : : +- CometSort (18) + : : +- ReusedExchange (17) + : +- CometSort (57) + : +- CometColumnarExchange (56) + : +- * HashAggregate (55) + : +- * CometColumnarToRow (54) + : +- CometColumnarExchange (53) + : +- * HashAggregate (52) + : +- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Project (43) + : : +- Window (42) + : : +- * CometColumnarToRow (41) + : : +- CometSort (40) + : : +- CometExchange (39) + : : +- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometFilter (32) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) + : : +- ReusedExchange (33) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- Window (47) + : +- * CometColumnarToRow (46) + : +- CometSort (45) + : +- ReusedExchange (44) + +- BroadcastExchange (70) + +- * Project (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- ReusedExchange (65) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,274 +139,310 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) CometWindowExec +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11], [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) CometProject +(15) Window +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] + +(16) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] -Arguments: [item_sk#9, d_date#6, sumws#10, rk#11], [item_sk#9, d_date#6, sumws#10, rk#11] -(16) ReusedExchange [Reuses operator id: 12] +(17) ReusedExchange [Reuses operator id: 12] Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(17) CometSort +(18) CometSort Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] -(18) CometWindowExec +(19) CometColumnarToRow [codegen id : 2] +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] + +(20) Window Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14], [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] +Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] -(19) CometProject +(21) Project [codegen id : 3] +Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] -Arguments: [item_sk#15, sumws#16, rk#14], [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] -(20) CometBroadcastExchange +(22) BroadcastExchange Input [3]: [item_sk#15, sumws#16, rk#14] -Arguments: [item_sk#15, sumws#16, rk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(21) CometBroadcastHashJoin -Left output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] -Right output [3]: [item_sk#15, sumws#16, rk#14] -Arguments: [item_sk#9], [item_sk#15], Inner, (rk#11 >= rk#14), BuildRight +(23) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [item_sk#9] +Right keys [1]: [item_sk#15] +Join type: Inner +Join condition: (rk#11 >= rk#14) -(22) CometProject +(24) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] -Arguments: [item_sk#9, d_date#6, sumws#10, sumws#16], [item_sk#9, d_date#6, sumws#10, sumws#16] -(23) CometHashAggregate +(25) HashAggregate [codegen id : 4] Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] Keys [3]: [item_sk#9, d_date#6, sumws#10] Functions [1]: [partial_sum(sumws#16)] +Aggregate Attributes [2]: [sum#17, isEmpty#18] +Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(26) CometColumnarExchange +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#17, isEmpty#18] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(27) CometColumnarToRow [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -(25) CometHashAggregate -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#17, isEmpty#18] +(28) HashAggregate [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] Keys [3]: [item_sk#9, d_date#6, sumws#10] Functions [1]: [sum(sumws#16)] +Aggregate Attributes [1]: [sum(sumws#16)#21] +Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] -(26) CometExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#19] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(29) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(27) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#19] -Arguments: [item_sk#9, d_date#6, cume_sales#19], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(30) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(29) CometFilter -Input [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_item_sk#20) - -(30) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#24, d_date#25] - -(31) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Right output [2]: [d_date_sk#24, d_date#25] -Arguments: [ss_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight - -(32) CometProject -Input [5]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#24, d_date#25] -Arguments: [ss_item_sk#20, ss_sales_price#21, d_date#25], [ss_item_sk#20, ss_sales_price#21, d_date#25] - -(33) CometHashAggregate -Input [3]: [ss_item_sk#20, ss_sales_price#21, d_date#25] -Keys [2]: [ss_item_sk#20, d_date#25] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] - -(34) CometExchange -Input [3]: [ss_item_sk#20, d_date#25, sum#26] -Arguments: hashpartitioning(ss_item_sk#20, d_date#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(35) CometHashAggregate -Input [3]: [ss_item_sk#20, d_date#25, sum#26] -Keys [2]: [ss_item_sk#20, d_date#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] - -(36) CometExchange -Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] -Arguments: hashpartitioning(ss_item_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(37) CometSort -Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] -Arguments: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20], [ss_item_sk#20 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] - -(38) CometWindowExec -Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] -Arguments: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20, rk#29], [row_number() windowspecdefinition(ss_item_sk#20, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#29], [ss_item_sk#20], [d_date#25 ASC NULLS FIRST] - -(39) CometProject -Input [5]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20, rk#29] -Arguments: [item_sk#27, d_date#25, sumss#28, rk#29], [item_sk#27, d_date#25, sumss#28, rk#29] - -(40) ReusedExchange [Reuses operator id: 36] -Output [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] - -(41) CometSort -Input [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] -Arguments: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31], [ss_item_sk#31 ASC NULLS FIRST, d_date#30 ASC NULLS FIRST] - -(42) CometWindowExec -Input [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] -Arguments: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31, rk#32], [row_number() windowspecdefinition(ss_item_sk#31, d_date#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#31], [d_date#30 ASC NULLS FIRST] - -(43) CometProject -Input [5]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31, rk#32] -Arguments: [item_sk#33, sumss#34, rk#32], [item_sk#27 AS item_sk#33, sumss#28 AS sumss#34, rk#32] - -(44) CometBroadcastExchange -Input [3]: [item_sk#33, sumss#34, rk#32] -Arguments: [item_sk#33, sumss#34, rk#32] - -(45) CometBroadcastHashJoin -Left output [4]: [item_sk#27, d_date#25, sumss#28, rk#29] -Right output [3]: [item_sk#33, sumss#34, rk#32] -Arguments: [item_sk#27], [item_sk#33], Inner, (rk#29 >= rk#32), BuildRight - -(46) CometProject -Input [7]: [item_sk#27, d_date#25, sumss#28, rk#29, item_sk#33, sumss#34, rk#32] -Arguments: [item_sk#27, d_date#25, sumss#28, sumss#34], [item_sk#27, d_date#25, sumss#28, sumss#34] - -(47) CometHashAggregate -Input [4]: [item_sk#27, d_date#25, sumss#28, sumss#34] -Keys [3]: [item_sk#27, d_date#25, sumss#28] -Functions [1]: [partial_sum(sumss#34)] - -(48) CometExchange -Input [5]: [item_sk#27, d_date#25, sumss#28, sum#35, isEmpty#36] -Arguments: hashpartitioning(item_sk#27, d_date#25, sumss#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(49) CometHashAggregate -Input [5]: [item_sk#27, d_date#25, sumss#28, sum#35, isEmpty#36] -Keys [3]: [item_sk#27, d_date#25, sumss#28] -Functions [1]: [sum(sumss#34)] - -(50) CometExchange -Input [3]: [item_sk#27, d_date#25, cume_sales#37] -Arguments: hashpartitioning(item_sk#27, d_date#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(51) CometSort -Input [3]: [item_sk#27, d_date#25, cume_sales#37] -Arguments: [item_sk#27, d_date#25, cume_sales#37], [item_sk#27 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#19] -Right output [3]: [item_sk#27, d_date#25, cume_sales#37] -Arguments: [item_sk#9, d_date#6], [item_sk#27, d_date#25], FullOuter - -(53) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#19, item_sk#27, d_date#25, cume_sales#37] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#27 END) - -(54) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#19, item_sk#27, d_date#25, cume_sales#37] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#27 END AS item_sk#38, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#25 END AS d_date#39, cume_sales#19 AS web_sales#40, cume_sales#37 AS store_sales#41] - -(55) CometExchange -Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Arguments: hashpartitioning(item_sk#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(56) CometSort -Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST] - -(57) CometWindowExec -Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42], [row_number() windowspecdefinition(item_sk#38, d_date#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [item_sk#38], [d_date#39 ASC NULLS FIRST] - -(58) ReusedExchange [Reuses operator id: 55] -Output [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +(32) CometFilter +Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_item_sk#23) + +(33) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#27, d_date#28] + +(34) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Right output [2]: [d_date_sk#27, d_date#28] +Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight + +(35) CometProject +Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] +Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] + +(36) CometHashAggregate +Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] + +(37) CometExchange +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(38) CometHashAggregate +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] + +(39) CometExchange +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(59) CometSort -Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST] +(40) CometSort +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(60) CometWindowExec -Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#43], [row_number() windowspecdefinition(item_sk#38, d_date#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [item_sk#38], [d_date#39 ASC NULLS FIRST] +(41) CometColumnarToRow [codegen id : 6] +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -(61) CometProject -Input [5]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#43] -Arguments: [item_sk#44, web_sales#45, store_sales#46, rk#43], [item_sk#38 AS item_sk#44, web_sales#40 AS web_sales#45, store_sales#41 AS store_sales#46, rk#43] +(42) Window +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] -(62) CometBroadcastExchange -Input [4]: [item_sk#44, web_sales#45, store_sales#46, rk#43] -Arguments: [item_sk#44, web_sales#45, store_sales#46, rk#43] +(43) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] +Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] -(63) CometBroadcastHashJoin -Left output [5]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42] -Right output [4]: [item_sk#44, web_sales#45, store_sales#46, rk#43] -Arguments: [item_sk#38], [item_sk#44], Inner, (rk#42 >= rk#43), BuildRight +(44) ReusedExchange [Reuses operator id: 39] +Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(64) CometProject -Input [9]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42, item_sk#44, web_sales#45, store_sales#46, rk#43] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46], [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46] +(45) CometSort +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] -(65) CometHashAggregate -Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46] -Keys [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Functions [2]: [partial_max(web_sales#45), partial_max(store_sales#46)] - -(66) CometHashAggregate -Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, max#47, max#48] -Keys [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Functions [2]: [max(web_sales#45), max(store_sales#46)] - -(67) CometFilter -Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] -Condition : ((isnotnull(web_cumulative#49) AND isnotnull(store_cumulative#50)) AND (web_cumulative#49 > store_cumulative#50)) - -(68) CometTakeOrderedAndProject -Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#38 ASC NULLS FIRST,d_date#39 ASC NULLS FIRST], output=[item_sk#38,d_date#39,web_sales#40,store_sales#41,web_cumulative#49,store_cumulative#50]), [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50], 100, 0, [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST], [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] - -(69) CometColumnarToRow [codegen id : 1] -Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] +(46) CometColumnarToRow [codegen id : 7] +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] + +(47) Window +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] + +(48) Project [codegen id : 8] +Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] +Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] + +(49) BroadcastExchange +Input [3]: [item_sk#36, sumss#37, rk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +(50) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [item_sk#30] +Right keys [1]: [item_sk#36] +Join type: Inner +Join condition: (rk#32 >= rk#35) + +(51) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] + +(52) HashAggregate [codegen id : 9] +Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [partial_sum(sumss#37)] +Aggregate Attributes [2]: [sum#38, isEmpty#39] +Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(53) CometColumnarExchange +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(54) CometColumnarToRow [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(55) HashAggregate [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [sum(sumss#37)] +Aggregate Attributes [1]: [sum(sumss#37)#42] +Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] + +(56) CometColumnarExchange +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(57) CometSort +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] + +(58) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#22] +Right output [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter + +(59) CometFilter +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) + +(60) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] + +(61) CometExchange +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(62) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] + +(63) CometColumnarToRow [codegen id : 11] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(64) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] + +(65) ReusedExchange [Reuses operator id: 61] +Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(66) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] + +(67) CometColumnarToRow [codegen id : 22] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(68) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] + +(69) Project [codegen id : 23] +Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] +Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] + +(70) BroadcastExchange +Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] + +(71) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [item_sk#44] +Right keys [1]: [item_sk#50] +Join type: Inner +Join condition: (rk#48 >= rk#49) + +(72) Project [codegen id : 24] +Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] + +(73) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] +Aggregate Attributes [2]: [max#53, max#54] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] + +(74) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [max(web_sales#51), max(store_sales#52)] +Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] + +(75) Filter [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) + +(76) TakeOrderedAndProject +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometProject (72) - +- CometFilter (71) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (70) +BroadcastExchange (81) ++- * CometColumnarToRow (80) + +- CometProject (79) + +- CometFilter (78) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(71) CometFilter +(78) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(72) CometProject +(79) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(73) CometColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(74) BroadcastExchange +(81) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt index 8aed2793f6..15a87f66b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/extended.txt @@ -1,203 +1,216 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometWindowExec - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- HashAggregate + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- Window [COMET: Window expressions are not supported] + : : : +- CometColumnarToRow + : : : +- CometSort + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort : +- CometExchange : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort +- CometExchange +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 184 out of 196 eligible operators (93%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt index 09f0ec24d0..b3013059b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a.native_iceberg_compat/simplified.txt @@ -1,80 +1,116 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometHashAggregate [max,max] [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative,max(web_sales),max(store_sales)] - CometHashAggregate [web_sales,store_sales] [item_sk,d_date,web_sales,store_sales,max,max] - CometProject [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - CometBroadcastHashJoin [item_sk,d_date,web_sales,store_sales,rk,item_sk,web_sales,store_sales,rk] - CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #2 - CometHashAggregate [sumws,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumws)] - CometExchange [item_sk,d_date,sumws] #3 - CometHashAggregate [sumws] [item_sk,d_date,sumws,sum,isEmpty] - CometProject [item_sk,d_date,sumws,sumws] - CometBroadcastHashJoin [item_sk,d_date,sumws,rk,item_sk,sumws,rk] - CometProject [item_sk,d_date,sumws,rk] - CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [item_sk,sumws,rk] #8 - CometProject [item_sk,sumws] [item_sk,sumws,rk] - CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #9 - CometHashAggregate [sumss,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumss)] - CometExchange [item_sk,d_date,sumss] #10 - CometHashAggregate [sumss] [item_sk,d_date,sumss,sum,isEmpty] - CometProject [item_sk,d_date,sumss,sumss] - CometBroadcastHashJoin [item_sk,d_date,sumss,rk,item_sk,sumss,rk] - CometProject [item_sk,d_date,sumss,rk] - CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 - CometBroadcastExchange [item_sk,sumss,rk] #13 - CometProject [item_sk,sumss] [item_sk,sumss,rk] - CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - CometBroadcastExchange [item_sk,web_sales,store_sales,rk] #14 - CometProject [item_sk,web_sales,store_sales] [item_sk,web_sales,store_sales,rk] - CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (24) + Filter [web_cumulative,store_cumulative] + HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (11) + CometColumnarToRow + InputAdapter CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (5) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (4) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometExchange [ws_item_sk] #4 + CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #9 + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + WholeStageCodegen (9) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (23) + Project [item_sk,web_sales,store_sales,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (22) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt index ce43cefb57..f8a95a6db8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/explain.txt @@ -1,73 +1,80 @@ == Physical Plan == -* CometColumnarToRow (69) -+- CometTakeOrderedAndProject (68) - +- CometFilter (67) - +- CometHashAggregate (66) - +- CometHashAggregate (65) - +- CometProject (64) - +- CometBroadcastHashJoin (63) - :- CometWindowExec (57) - : +- CometSort (56) - : +- CometExchange (55) - : +- CometProject (54) - : +- CometFilter (53) - : +- CometSortMergeJoin (52) - : :- CometSort (27) - : : +- CometExchange (26) - : : +- CometHashAggregate (25) - : : +- CometExchange (24) - : : +- CometHashAggregate (23) - : : +- CometProject (22) - : : +- CometBroadcastHashJoin (21) - : : :- CometProject (15) - : : : +- CometWindowExec (14) - : : : +- CometSort (13) - : : : +- CometExchange (12) - : : : +- CometHashAggregate (11) - : : : +- CometExchange (10) - : : : +- CometHashAggregate (9) - : : : +- CometProject (8) - : : : +- CometBroadcastHashJoin (7) - : : : :- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) - : : : +- CometBroadcastExchange (6) - : : : +- CometProject (5) - : : : +- CometFilter (4) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : : +- CometBroadcastExchange (20) - : : +- CometProject (19) - : : +- CometWindowExec (18) - : : +- CometSort (17) - : : +- ReusedExchange (16) - : +- CometSort (51) - : +- CometExchange (50) - : +- CometHashAggregate (49) - : +- CometExchange (48) - : +- CometHashAggregate (47) - : +- CometProject (46) - : +- CometBroadcastHashJoin (45) - : :- CometProject (39) - : : +- CometWindowExec (38) - : : +- CometSort (37) - : : +- CometExchange (36) - : : +- CometHashAggregate (35) - : : +- CometExchange (34) - : : +- CometHashAggregate (33) - : : +- CometProject (32) - : : +- CometBroadcastHashJoin (31) - : : :- CometFilter (29) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (28) - : : +- ReusedExchange (30) - : +- CometBroadcastExchange (44) - : +- CometProject (43) - : +- CometWindowExec (42) - : +- CometSort (41) - : +- ReusedExchange (40) - +- CometBroadcastExchange (62) - +- CometProject (61) - +- CometWindowExec (60) - +- CometSort (59) - +- ReusedExchange (58) +TakeOrderedAndProject (76) ++- * Filter (75) + +- * HashAggregate (74) + +- * HashAggregate (73) + +- * Project (72) + +- * BroadcastHashJoin Inner BuildRight (71) + :- Window (64) + : +- * CometColumnarToRow (63) + : +- CometSort (62) + : +- CometExchange (61) + : +- CometProject (60) + : +- CometFilter (59) + : +- CometSortMergeJoin (58) + : :- CometSort (30) + : : +- CometColumnarExchange (29) + : : +- * HashAggregate (28) + : : +- * CometColumnarToRow (27) + : : +- CometColumnarExchange (26) + : : +- * HashAggregate (25) + : : +- * Project (24) + : : +- * BroadcastHashJoin Inner BuildRight (23) + : : :- * Project (16) + : : : +- Window (15) + : : : +- * CometColumnarToRow (14) + : : : +- CometSort (13) + : : : +- CometExchange (12) + : : : +- CometHashAggregate (11) + : : : +- CometExchange (10) + : : : +- CometHashAggregate (9) + : : : +- CometProject (8) + : : : +- CometBroadcastHashJoin (7) + : : : :- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (1) + : : : +- CometBroadcastExchange (6) + : : : +- CometProject (5) + : : : +- CometFilter (4) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : : +- BroadcastExchange (22) + : : +- * Project (21) + : : +- Window (20) + : : +- * CometColumnarToRow (19) + : : +- CometSort (18) + : : +- ReusedExchange (17) + : +- CometSort (57) + : +- CometColumnarExchange (56) + : +- * HashAggregate (55) + : +- * CometColumnarToRow (54) + : +- CometColumnarExchange (53) + : +- * HashAggregate (52) + : +- * Project (51) + : +- * BroadcastHashJoin Inner BuildRight (50) + : :- * Project (43) + : : +- Window (42) + : : +- * CometColumnarToRow (41) + : : +- CometSort (40) + : : +- CometExchange (39) + : : +- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometFilter (32) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (31) + : : +- ReusedExchange (33) + : +- BroadcastExchange (49) + : +- * Project (48) + : +- Window (47) + : +- * CometColumnarToRow (46) + : +- CometSort (45) + : +- ReusedExchange (44) + +- BroadcastExchange (70) + +- * Project (69) + +- Window (68) + +- * CometColumnarToRow (67) + +- CometSort (66) + +- ReusedExchange (65) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales @@ -132,274 +139,310 @@ Arguments: hashpartitioning(ws_item_sk#1, 5), ENSURE_REQUIREMENTS, CometNativeSh Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1], [ws_item_sk#1 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(14) CometWindowExec +(14) CometColumnarToRow [codegen id : 1] Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] -Arguments: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11], [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] -(15) CometProject +(15) Window +Input [4]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1] +Arguments: [row_number() windowspecdefinition(ws_item_sk#1, d_date#6 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#11], [ws_item_sk#1], [d_date#6 ASC NULLS FIRST] + +(16) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] Input [5]: [item_sk#9, d_date#6, sumws#10, ws_item_sk#1, rk#11] -Arguments: [item_sk#9, d_date#6, sumws#10, rk#11], [item_sk#9, d_date#6, sumws#10, rk#11] -(16) ReusedExchange [Reuses operator id: 12] +(17) ReusedExchange [Reuses operator id: 12] Output [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -(17) CometSort +(18) CometSort Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13], [ws_item_sk#13 ASC NULLS FIRST, d_date#12 ASC NULLS FIRST] -(18) CometWindowExec +(19) CometColumnarToRow [codegen id : 2] +Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] + +(20) Window Input [4]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13] -Arguments: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14], [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] +Arguments: [row_number() windowspecdefinition(ws_item_sk#13, d_date#12 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#14], [ws_item_sk#13], [d_date#12 ASC NULLS FIRST] -(19) CometProject +(21) Project [codegen id : 3] +Output [3]: [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] Input [5]: [item_sk#9, d_date#12, sumws#10, ws_item_sk#13, rk#14] -Arguments: [item_sk#15, sumws#16, rk#14], [item_sk#9 AS item_sk#15, sumws#10 AS sumws#16, rk#14] -(20) CometBroadcastExchange +(22) BroadcastExchange Input [3]: [item_sk#15, sumws#16, rk#14] -Arguments: [item_sk#15, sumws#16, rk#14] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(21) CometBroadcastHashJoin -Left output [4]: [item_sk#9, d_date#6, sumws#10, rk#11] -Right output [3]: [item_sk#15, sumws#16, rk#14] -Arguments: [item_sk#9], [item_sk#15], Inner, (rk#11 >= rk#14), BuildRight +(23) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [item_sk#9] +Right keys [1]: [item_sk#15] +Join type: Inner +Join condition: (rk#11 >= rk#14) -(22) CometProject +(24) Project [codegen id : 4] +Output [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] Input [7]: [item_sk#9, d_date#6, sumws#10, rk#11, item_sk#15, sumws#16, rk#14] -Arguments: [item_sk#9, d_date#6, sumws#10, sumws#16], [item_sk#9, d_date#6, sumws#10, sumws#16] -(23) CometHashAggregate +(25) HashAggregate [codegen id : 4] Input [4]: [item_sk#9, d_date#6, sumws#10, sumws#16] Keys [3]: [item_sk#9, d_date#6, sumws#10] Functions [1]: [partial_sum(sumws#16)] +Aggregate Attributes [2]: [sum#17, isEmpty#18] +Results [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] + +(26) CometColumnarExchange +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] +Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(24) CometExchange -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#17, isEmpty#18] -Arguments: hashpartitioning(item_sk#9, d_date#6, sumws#10, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(27) CometColumnarToRow [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] -(25) CometHashAggregate -Input [5]: [item_sk#9, d_date#6, sumws#10, sum#17, isEmpty#18] +(28) HashAggregate [codegen id : 5] +Input [5]: [item_sk#9, d_date#6, sumws#10, sum#19, isEmpty#20] Keys [3]: [item_sk#9, d_date#6, sumws#10] Functions [1]: [sum(sumws#16)] +Aggregate Attributes [1]: [sum(sumws#16)#21] +Results [3]: [item_sk#9, d_date#6, sum(sumws#16)#21 AS cume_sales#22] -(26) CometExchange -Input [3]: [item_sk#9, d_date#6, cume_sales#19] -Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(29) CometColumnarExchange +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: hashpartitioning(item_sk#9, d_date#6, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(27) CometSort -Input [3]: [item_sk#9, d_date#6, cume_sales#19] -Arguments: [item_sk#9, d_date#6, cume_sales#19], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] +(30) CometSort +Input [3]: [item_sk#9, d_date#6, cume_sales#22] +Arguments: [item_sk#9, d_date#6, cume_sales#22], [item_sk#9 ASC NULLS FIRST, d_date#6 ASC NULLS FIRST] -(28) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales -Output [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] +(31) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +Output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ss_sold_date_sk#22), dynamicpruningexpression(ss_sold_date_sk#22 IN dynamicpruning#23)] +PartitionFilters: [isnotnull(ss_sold_date_sk#25), dynamicpruningexpression(ss_sold_date_sk#25 IN dynamicpruning#26)] PushedFilters: [IsNotNull(ss_item_sk)] ReadSchema: struct -(29) CometFilter -Input [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Condition : isnotnull(ss_item_sk#20) - -(30) ReusedExchange [Reuses operator id: 6] -Output [2]: [d_date_sk#24, d_date#25] - -(31) CometBroadcastHashJoin -Left output [3]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22] -Right output [2]: [d_date_sk#24, d_date#25] -Arguments: [ss_sold_date_sk#22], [d_date_sk#24], Inner, BuildRight - -(32) CometProject -Input [5]: [ss_item_sk#20, ss_sales_price#21, ss_sold_date_sk#22, d_date_sk#24, d_date#25] -Arguments: [ss_item_sk#20, ss_sales_price#21, d_date#25], [ss_item_sk#20, ss_sales_price#21, d_date#25] - -(33) CometHashAggregate -Input [3]: [ss_item_sk#20, ss_sales_price#21, d_date#25] -Keys [2]: [ss_item_sk#20, d_date#25] -Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#21))] - -(34) CometExchange -Input [3]: [ss_item_sk#20, d_date#25, sum#26] -Arguments: hashpartitioning(ss_item_sk#20, d_date#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] - -(35) CometHashAggregate -Input [3]: [ss_item_sk#20, d_date#25, sum#26] -Keys [2]: [ss_item_sk#20, d_date#25] -Functions [1]: [sum(UnscaledValue(ss_sales_price#21))] - -(36) CometExchange -Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] -Arguments: hashpartitioning(ss_item_sk#20, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] - -(37) CometSort -Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] -Arguments: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20], [ss_item_sk#20 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] - -(38) CometWindowExec -Input [4]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20] -Arguments: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20, rk#29], [row_number() windowspecdefinition(ss_item_sk#20, d_date#25 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#29], [ss_item_sk#20], [d_date#25 ASC NULLS FIRST] - -(39) CometProject -Input [5]: [item_sk#27, d_date#25, sumss#28, ss_item_sk#20, rk#29] -Arguments: [item_sk#27, d_date#25, sumss#28, rk#29], [item_sk#27, d_date#25, sumss#28, rk#29] - -(40) ReusedExchange [Reuses operator id: 36] -Output [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] - -(41) CometSort -Input [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] -Arguments: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31], [ss_item_sk#31 ASC NULLS FIRST, d_date#30 ASC NULLS FIRST] - -(42) CometWindowExec -Input [4]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31] -Arguments: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31, rk#32], [row_number() windowspecdefinition(ss_item_sk#31, d_date#30 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#31], [d_date#30 ASC NULLS FIRST] - -(43) CometProject -Input [5]: [item_sk#27, d_date#30, sumss#28, ss_item_sk#31, rk#32] -Arguments: [item_sk#33, sumss#34, rk#32], [item_sk#27 AS item_sk#33, sumss#28 AS sumss#34, rk#32] - -(44) CometBroadcastExchange -Input [3]: [item_sk#33, sumss#34, rk#32] -Arguments: [item_sk#33, sumss#34, rk#32] - -(45) CometBroadcastHashJoin -Left output [4]: [item_sk#27, d_date#25, sumss#28, rk#29] -Right output [3]: [item_sk#33, sumss#34, rk#32] -Arguments: [item_sk#27], [item_sk#33], Inner, (rk#29 >= rk#32), BuildRight - -(46) CometProject -Input [7]: [item_sk#27, d_date#25, sumss#28, rk#29, item_sk#33, sumss#34, rk#32] -Arguments: [item_sk#27, d_date#25, sumss#28, sumss#34], [item_sk#27, d_date#25, sumss#28, sumss#34] - -(47) CometHashAggregate -Input [4]: [item_sk#27, d_date#25, sumss#28, sumss#34] -Keys [3]: [item_sk#27, d_date#25, sumss#28] -Functions [1]: [partial_sum(sumss#34)] - -(48) CometExchange -Input [5]: [item_sk#27, d_date#25, sumss#28, sum#35, isEmpty#36] -Arguments: hashpartitioning(item_sk#27, d_date#25, sumss#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] - -(49) CometHashAggregate -Input [5]: [item_sk#27, d_date#25, sumss#28, sum#35, isEmpty#36] -Keys [3]: [item_sk#27, d_date#25, sumss#28] -Functions [1]: [sum(sumss#34)] - -(50) CometExchange -Input [3]: [item_sk#27, d_date#25, cume_sales#37] -Arguments: hashpartitioning(item_sk#27, d_date#25, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] - -(51) CometSort -Input [3]: [item_sk#27, d_date#25, cume_sales#37] -Arguments: [item_sk#27, d_date#25, cume_sales#37], [item_sk#27 ASC NULLS FIRST, d_date#25 ASC NULLS FIRST] - -(52) CometSortMergeJoin -Left output [3]: [item_sk#9, d_date#6, cume_sales#19] -Right output [3]: [item_sk#27, d_date#25, cume_sales#37] -Arguments: [item_sk#9, d_date#6], [item_sk#27, d_date#25], FullOuter - -(53) CometFilter -Input [6]: [item_sk#9, d_date#6, cume_sales#19, item_sk#27, d_date#25, cume_sales#37] -Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#27 END) - -(54) CometProject -Input [6]: [item_sk#9, d_date#6, cume_sales#19, item_sk#27, d_date#25, cume_sales#37] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#27 END AS item_sk#38, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#25 END AS d_date#39, cume_sales#19 AS web_sales#40, cume_sales#37 AS store_sales#41] - -(55) CometExchange -Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Arguments: hashpartitioning(item_sk#38, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(56) CometSort -Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST] - -(57) CometWindowExec -Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42], [row_number() windowspecdefinition(item_sk#38, d_date#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#42], [item_sk#38], [d_date#39 ASC NULLS FIRST] - -(58) ReusedExchange [Reuses operator id: 55] -Output [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] +(32) CometFilter +Input [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Condition : isnotnull(ss_item_sk#23) + +(33) ReusedExchange [Reuses operator id: 6] +Output [2]: [d_date_sk#27, d_date#28] + +(34) CometBroadcastHashJoin +Left output [3]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25] +Right output [2]: [d_date_sk#27, d_date#28] +Arguments: [ss_sold_date_sk#25], [d_date_sk#27], Inner, BuildRight + +(35) CometProject +Input [5]: [ss_item_sk#23, ss_sales_price#24, ss_sold_date_sk#25, d_date_sk#27, d_date#28] +Arguments: [ss_item_sk#23, ss_sales_price#24, d_date#28], [ss_item_sk#23, ss_sales_price#24, d_date#28] + +(36) CometHashAggregate +Input [3]: [ss_item_sk#23, ss_sales_price#24, d_date#28] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [partial_sum(UnscaledValue(ss_sales_price#24))] + +(37) CometExchange +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Arguments: hashpartitioning(ss_item_sk#23, d_date#28, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] + +(38) CometHashAggregate +Input [3]: [ss_item_sk#23, d_date#28, sum#29] +Keys [2]: [ss_item_sk#23, d_date#28] +Functions [1]: [sum(UnscaledValue(ss_sales_price#24))] + +(39) CometExchange +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: hashpartitioning(ss_item_sk#23, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(59) CometSort -Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41], [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST] +(40) CometSort +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23], [ss_item_sk#23 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] -(60) CometWindowExec -Input [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#43], [row_number() windowspecdefinition(item_sk#38, d_date#39 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#43], [item_sk#38], [d_date#39 ASC NULLS FIRST] +(41) CometColumnarToRow [codegen id : 6] +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] -(61) CometProject -Input [5]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#43] -Arguments: [item_sk#44, web_sales#45, store_sales#46, rk#43], [item_sk#38 AS item_sk#44, web_sales#40 AS web_sales#45, store_sales#41 AS store_sales#46, rk#43] +(42) Window +Input [4]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23] +Arguments: [row_number() windowspecdefinition(ss_item_sk#23, d_date#28 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#32], [ss_item_sk#23], [d_date#28 ASC NULLS FIRST] -(62) CometBroadcastExchange -Input [4]: [item_sk#44, web_sales#45, store_sales#46, rk#43] -Arguments: [item_sk#44, web_sales#45, store_sales#46, rk#43] +(43) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, rk#32] +Input [5]: [item_sk#30, d_date#28, sumss#31, ss_item_sk#23, rk#32] -(63) CometBroadcastHashJoin -Left output [5]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42] -Right output [4]: [item_sk#44, web_sales#45, store_sales#46, rk#43] -Arguments: [item_sk#38], [item_sk#44], Inner, (rk#42 >= rk#43), BuildRight +(44) ReusedExchange [Reuses operator id: 39] +Output [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] -(64) CometProject -Input [9]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, rk#42, item_sk#44, web_sales#45, store_sales#46, rk#43] -Arguments: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46], [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46] +(45) CometSort +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34], [ss_item_sk#34 ASC NULLS FIRST, d_date#33 ASC NULLS FIRST] -(65) CometHashAggregate -Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_sales#45, store_sales#46] -Keys [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Functions [2]: [partial_max(web_sales#45), partial_max(store_sales#46)] - -(66) CometHashAggregate -Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, max#47, max#48] -Keys [4]: [item_sk#38, d_date#39, web_sales#40, store_sales#41] -Functions [2]: [max(web_sales#45), max(store_sales#46)] - -(67) CometFilter -Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] -Condition : ((isnotnull(web_cumulative#49) AND isnotnull(store_cumulative#50)) AND (web_cumulative#49 > store_cumulative#50)) - -(68) CometTakeOrderedAndProject -Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[item_sk#38 ASC NULLS FIRST,d_date#39 ASC NULLS FIRST], output=[item_sk#38,d_date#39,web_sales#40,store_sales#41,web_cumulative#49,store_cumulative#50]), [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50], 100, 0, [item_sk#38 ASC NULLS FIRST, d_date#39 ASC NULLS FIRST], [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] - -(69) CometColumnarToRow [codegen id : 1] -Input [6]: [item_sk#38, d_date#39, web_sales#40, store_sales#41, web_cumulative#49, store_cumulative#50] +(46) CometColumnarToRow [codegen id : 7] +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] + +(47) Window +Input [4]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34] +Arguments: [row_number() windowspecdefinition(ss_item_sk#34, d_date#33 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#35], [ss_item_sk#34], [d_date#33 ASC NULLS FIRST] + +(48) Project [codegen id : 8] +Output [3]: [item_sk#30 AS item_sk#36, sumss#31 AS sumss#37, rk#35] +Input [5]: [item_sk#30, d_date#33, sumss#31, ss_item_sk#34, rk#35] + +(49) BroadcastExchange +Input [3]: [item_sk#36, sumss#37, rk#35] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] + +(50) BroadcastHashJoin [codegen id : 9] +Left keys [1]: [item_sk#30] +Right keys [1]: [item_sk#36] +Join type: Inner +Join condition: (rk#32 >= rk#35) + +(51) Project [codegen id : 9] +Output [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Input [7]: [item_sk#30, d_date#28, sumss#31, rk#32, item_sk#36, sumss#37, rk#35] + +(52) HashAggregate [codegen id : 9] +Input [4]: [item_sk#30, d_date#28, sumss#31, sumss#37] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [partial_sum(sumss#37)] +Aggregate Attributes [2]: [sum#38, isEmpty#39] +Results [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(53) CometColumnarExchange +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Arguments: hashpartitioning(item_sk#30, d_date#28, sumss#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=9] + +(54) CometColumnarToRow [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] + +(55) HashAggregate [codegen id : 10] +Input [5]: [item_sk#30, d_date#28, sumss#31, sum#40, isEmpty#41] +Keys [3]: [item_sk#30, d_date#28, sumss#31] +Functions [1]: [sum(sumss#37)] +Aggregate Attributes [1]: [sum(sumss#37)#42] +Results [3]: [item_sk#30, d_date#28, sum(sumss#37)#42 AS cume_sales#43] + +(56) CometColumnarExchange +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: hashpartitioning(item_sk#30, d_date#28, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] + +(57) CometSort +Input [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#30, d_date#28, cume_sales#43], [item_sk#30 ASC NULLS FIRST, d_date#28 ASC NULLS FIRST] + +(58) CometSortMergeJoin +Left output [3]: [item_sk#9, d_date#6, cume_sales#22] +Right output [3]: [item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#9, d_date#6], [item_sk#30, d_date#28], FullOuter + +(59) CometFilter +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Condition : isnotnull(CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END) + +(60) CometProject +Input [6]: [item_sk#9, d_date#6, cume_sales#22, item_sk#30, d_date#28, cume_sales#43] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [CASE WHEN isnotnull(item_sk#9) THEN item_sk#9 ELSE item_sk#30 END AS item_sk#44, CASE WHEN isnotnull(d_date#6) THEN d_date#6 ELSE d_date#28 END AS d_date#45, cume_sales#22 AS web_sales#46, cume_sales#43 AS store_sales#47] + +(61) CometExchange +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: hashpartitioning(item_sk#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(62) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] + +(63) CometColumnarToRow [codegen id : 11] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(64) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#48], [item_sk#44], [d_date#45 ASC NULLS FIRST] + +(65) ReusedExchange [Reuses operator id: 61] +Output [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(66) CometSort +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [item_sk#44, d_date#45, web_sales#46, store_sales#47], [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST] + +(67) CometColumnarToRow [codegen id : 22] +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] + +(68) Window +Input [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Arguments: [row_number() windowspecdefinition(item_sk#44, d_date#45 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#49], [item_sk#44], [d_date#45 ASC NULLS FIRST] + +(69) Project [codegen id : 23] +Output [4]: [item_sk#44 AS item_sk#50, web_sales#46 AS web_sales#51, store_sales#47 AS store_sales#52, rk#49] +Input [5]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#49] + +(70) BroadcastExchange +Input [4]: [item_sk#50, web_sales#51, store_sales#52, rk#49] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=12] + +(71) BroadcastHashJoin [codegen id : 24] +Left keys [1]: [item_sk#44] +Right keys [1]: [item_sk#50] +Join type: Inner +Join condition: (rk#48 >= rk#49) + +(72) Project [codegen id : 24] +Output [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Input [9]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, rk#48, item_sk#50, web_sales#51, store_sales#52, rk#49] + +(73) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_sales#51, store_sales#52] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [partial_max(web_sales#51), partial_max(store_sales#52)] +Aggregate Attributes [2]: [max#53, max#54] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] + +(74) HashAggregate [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max#55, max#56] +Keys [4]: [item_sk#44, d_date#45, web_sales#46, store_sales#47] +Functions [2]: [max(web_sales#51), max(store_sales#52)] +Aggregate Attributes [2]: [max(web_sales#51)#57, max(store_sales#52)#58] +Results [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, max(web_sales#51)#57 AS web_cumulative#59, max(store_sales#52)#58 AS store_cumulative#60] + +(75) Filter [codegen id : 24] +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Condition : ((isnotnull(web_cumulative#59) AND isnotnull(store_cumulative#60)) AND (web_cumulative#59 > store_cumulative#60)) + +(76) TakeOrderedAndProject +Input [6]: [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] +Arguments: 100, [item_sk#44 ASC NULLS FIRST, d_date#45 ASC NULLS FIRST], [item_sk#44, d_date#45, web_sales#46, store_sales#47, web_cumulative#59, store_cumulative#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ws_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (74) -+- * CometColumnarToRow (73) - +- CometProject (72) - +- CometFilter (71) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (70) +BroadcastExchange (81) ++- * CometColumnarToRow (80) + +- CometProject (79) + +- CometFilter (78) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (77) -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(77) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(71) CometFilter +(78) CometFilter Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Condition : (((isnotnull(d_month_seq#7) AND (d_month_seq#7 >= 1212)) AND (d_month_seq#7 <= 1223)) AND isnotnull(d_date_sk#5)) -(72) CometProject +(79) CometProject Input [3]: [d_date_sk#5, d_date#6, d_month_seq#7] Arguments: [d_date_sk#5, d_date#6], [d_date_sk#5, d_date#6] -(73) CometColumnarToRow [codegen id : 1] +(80) CometColumnarToRow [codegen id : 1] Input [2]: [d_date_sk#5, d_date#6] -(74) BroadcastExchange +(81) BroadcastExchange Input [2]: [d_date_sk#5, d_date#6] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=10] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13] -Subquery:2 Hosting operator id = 28 Hosting Expression = ss_sold_date_sk#22 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 31 Hosting Expression = ss_sold_date_sk#25 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt index 8aed2793f6..15a87f66b6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/extended.txt @@ -1,203 +1,216 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometHashAggregate - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometProject - : +- CometFilter - : +- CometSortMergeJoin - : :- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometWindowExec - : : : +- CometSort - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometExchange - : : : +- CometHashAggregate - : : : +- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- HashAggregate + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometProject + : +- CometFilter + : +- CometSortMergeJoin + : :- CometSort + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- CometColumnarToRow + : : +- CometColumnarExchange + : : +- HashAggregate + : : +- Project + : : +- BroadcastHashJoin + : : :- Project + : : : +- Window [COMET: Window expressions are not supported] + : : : +- CometColumnarToRow + : : : +- CometSort + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometExchange + : : : +- CometHashAggregate + : : : +- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- BroadcastExchange + : : +- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometSort + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometProject +- CometFilter +- CometSortMergeJoin :- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate + : +- CometColumnarExchange + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- Project + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort : +- CometExchange : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate + +- CometColumnarExchange + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort +- CometExchange +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : +- ReusedSubquery - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : +- ReusedSubquery + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 184 out of 196 eligible operators (93%). Final plan contains 5 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 138 out of 196 eligible operators (70%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt index 09f0ec24d0..b3013059b0 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q51a/simplified.txt @@ -1,80 +1,116 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometFilter [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] - CometHashAggregate [max,max] [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative,max(web_sales),max(store_sales)] - CometHashAggregate [web_sales,store_sales] [item_sk,d_date,web_sales,store_sales,max,max] - CometProject [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] - CometBroadcastHashJoin [item_sk,d_date,web_sales,store_sales,rk,item_sk,web_sales,store_sales,rk] - CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] - CometSort [item_sk,d_date,web_sales,store_sales] - CometExchange [item_sk] #1 - CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] - CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #2 - CometHashAggregate [sumws,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumws)] - CometExchange [item_sk,d_date,sumws] #3 - CometHashAggregate [sumws] [item_sk,d_date,sumws,sum,isEmpty] - CometProject [item_sk,d_date,sumws,sumws] - CometBroadcastHashJoin [item_sk,d_date,sumws,rk,item_sk,sumws,rk] - CometProject [item_sk,d_date,sumws,rk] - CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] - CometSort [item_sk,d_date,sumws,ws_item_sk] - CometExchange [ws_item_sk] #4 - CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] - CometExchange [ws_item_sk,d_date] #5 - CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] - CometProject [ws_item_sk,ws_sales_price,d_date] - CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] - CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #6 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [d_date_sk,d_date] #7 - CometProject [d_date_sk,d_date] - CometFilter [d_date_sk,d_date,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] - CometBroadcastExchange [item_sk,sumws,rk] #8 - CometProject [item_sk,sumws] [item_sk,sumws,rk] - CometWindowExec [item_sk,d_date,sumws,ws_item_sk,rk] - CometSort [item_sk,d_date,sumws,ws_item_sk] - ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 - CometSort [item_sk,d_date,cume_sales] - CometExchange [item_sk,d_date] #9 - CometHashAggregate [sumss,sum,isEmpty] [item_sk,d_date,cume_sales,sum(sumss)] - CometExchange [item_sk,d_date,sumss] #10 - CometHashAggregate [sumss] [item_sk,d_date,sumss,sum,isEmpty] - CometProject [item_sk,d_date,sumss,sumss] - CometBroadcastHashJoin [item_sk,d_date,sumss,rk,item_sk,sumss,rk] - CometProject [item_sk,d_date,sumss,rk] - CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] - CometSort [item_sk,d_date,sumss,ss_item_sk] - CometExchange [ss_item_sk] #11 - CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] - CometExchange [ss_item_sk,d_date] #12 - CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] - CometProject [ss_item_sk,ss_sales_price,d_date] - CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] - CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk,d_date] #7 - CometBroadcastExchange [item_sk,sumss,rk] #13 - CometProject [item_sk,sumss] [item_sk,sumss,rk] - CometWindowExec [item_sk,d_date,sumss,ss_item_sk,rk] - CometSort [item_sk,d_date,sumss,ss_item_sk] - ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 - CometBroadcastExchange [item_sk,web_sales,store_sales,rk] #14 - CometProject [item_sk,web_sales,store_sales] [item_sk,web_sales,store_sales,rk] - CometWindowExec [item_sk,d_date,web_sales,store_sales,rk] +TakeOrderedAndProject [item_sk,d_date,web_sales,store_sales,web_cumulative,store_cumulative] + WholeStageCodegen (24) + Filter [web_cumulative,store_cumulative] + HashAggregate [item_sk,d_date,web_sales,store_sales,max,max] [max(web_sales),max(store_sales),web_cumulative,store_cumulative,max,max] + HashAggregate [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] [max,max,max,max] + Project [item_sk,d_date,web_sales,store_sales,web_sales,store_sales] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (11) + CometColumnarToRow + InputAdapter CometSort [item_sk,d_date,web_sales,store_sales] - ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 + CometExchange [item_sk] #1 + CometProject [item_sk,item_sk,d_date,d_date,cume_sales,cume_sales] [item_sk,d_date,web_sales,store_sales] + CometFilter [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSortMergeJoin [item_sk,d_date,cume_sales,item_sk,d_date,cume_sales] + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #2 + WholeStageCodegen (5) + HashAggregate [item_sk,d_date,sumws,sum,isEmpty] [sum(sumws),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumws] #3 + WholeStageCodegen (4) + HashAggregate [item_sk,d_date,sumws,sumws] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumws,sumws] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + CometExchange [ws_item_sk] #4 + CometHashAggregate [sum] [item_sk,d_date,sumws,ws_item_sk,sum(UnscaledValue(ws_sales_price))] + CometExchange [ws_item_sk,d_date] #5 + CometHashAggregate [ws_sales_price] [ws_item_sk,d_date,sum] + CometProject [ws_item_sk,ws_sales_price,d_date] + CometBroadcastHashJoin [ws_item_sk,ws_sales_price,ws_sold_date_sk,d_date_sk,d_date] + CometFilter [ws_item_sk,ws_sales_price,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_sales_price,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #6 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + CometBroadcastExchange [d_date_sk,d_date] #7 + CometProject [d_date_sk,d_date] + CometFilter [d_date_sk,d_date,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date,d_month_seq] + InputAdapter + BroadcastExchange #8 + WholeStageCodegen (3) + Project [item_sk,sumws,rk] + InputAdapter + Window [ws_item_sk,d_date] + WholeStageCodegen (2) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumws,ws_item_sk] + ReusedExchange [item_sk,d_date,sumws,ws_item_sk] #4 + CometSort [item_sk,d_date,cume_sales] + CometColumnarExchange [item_sk,d_date] #9 + WholeStageCodegen (10) + HashAggregate [item_sk,d_date,sumss,sum,isEmpty] [sum(sumss),cume_sales,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [item_sk,d_date,sumss] #10 + WholeStageCodegen (9) + HashAggregate [item_sk,d_date,sumss,sumss] [sum,isEmpty,sum,isEmpty] + Project [item_sk,d_date,sumss,sumss] + BroadcastHashJoin [item_sk,item_sk,rk,rk] + Project [item_sk,d_date,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (6) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + CometExchange [ss_item_sk] #11 + CometHashAggregate [sum] [item_sk,d_date,sumss,ss_item_sk,sum(UnscaledValue(ss_sales_price))] + CometExchange [ss_item_sk,d_date] #12 + CometHashAggregate [ss_sales_price] [ss_item_sk,d_date,sum] + CometProject [ss_item_sk,ss_sales_price,d_date] + CometBroadcastHashJoin [ss_item_sk,ss_sales_price,ss_sold_date_sk,d_date_sk,d_date] + CometFilter [ss_item_sk,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_sales_price,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk,d_date] #7 + InputAdapter + BroadcastExchange #13 + WholeStageCodegen (8) + Project [item_sk,sumss,rk] + InputAdapter + Window [ss_item_sk,d_date] + WholeStageCodegen (7) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,sumss,ss_item_sk] + ReusedExchange [item_sk,d_date,sumss,ss_item_sk] #11 + InputAdapter + BroadcastExchange #14 + WholeStageCodegen (23) + Project [item_sk,web_sales,store_sales,rk] + InputAdapter + Window [item_sk,d_date] + WholeStageCodegen (22) + CometColumnarToRow + InputAdapter + CometSort [item_sk,d_date,web_sales,store_sales] + ReusedExchange [item_sk,d_date,web_sales,store_sales] #1 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt index c5215aa2f4..d1daee98c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/explain.txt @@ -1,46 +1,52 @@ == Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (28) - : : +- CometFilter (27) - : : +- CometWindowExec (26) - : : +- CometFilter (25) - : : +- CometWindowExec (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometWindowExec (33) - : +- CometSort (32) - : +- CometExchange (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - +- ReusedExchange (38) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -153,107 +159,131 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometWindowExec +(24) CometColumnarToRow [codegen id : 1] Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometFilter +(25) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(26) Filter [codegen id : 2] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(26) CometWindowExec +(27) Window Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20], [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(27) CometFilter +(28) Filter [codegen id : 7] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(28) CometProject +(29) Project [codegen id : 7] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -(29) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -(30) CometHashAggregate +(31) CometHashAggregate Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -(31) CometExchange +(32) CometExchange Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) CometSort +(33) CometSort Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(33) CometWindowExec +(34) CometColumnarToRow [codegen id : 3] +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] + +(35) Window Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26], [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(34) CometProject +(36) Project [codegen id : 4] +Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] -Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26], [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -(35) CometBroadcastExchange +(37) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(36) CometBroadcastHashJoin -Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Right output [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)], Inner, BuildRight +(38) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] +Join type: Inner +Join condition: None -(37) CometProject +(39) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -(38) ReusedExchange [Reuses operator id: 35] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +(40) ReusedExchange [Reuses operator id: 32] +Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] + +(41) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] + +(42) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] + +(43) Window +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] + +(44) Project [codegen id : 6] +Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] +Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] -(39) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Right output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#30, i_brand#31, cc_name#32, (rn#34 - 1)], Inner, BuildRight +(45) BroadcastExchange +Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(40) CometProject -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -Arguments: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#35, sum_sales#33 AS nsum#36] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] +Join type: Inner +Join condition: None -(41) CometTakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#20,sum_sales#17,psum#35,nsum#36]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], 100, 0, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +(47) Project [codegen id : 7] +Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -(42) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +(48) TakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(45) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(46) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt index 00de5f32a1..7bee0e61e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/extended.txt @@ -1,76 +1,78 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -100,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt index 7a51065baf..d655789fe6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.native_iceberg_compat/simplified.txt @@ -1,51 +1,71 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,cc_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] + WholeStageCodegen (7) + Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (2) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,cc_name] #8 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt index c5215aa2f4..d1daee98c1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt @@ -1,46 +1,52 @@ == Physical Plan == -* CometColumnarToRow (42) -+- CometTakeOrderedAndProject (41) - +- CometProject (40) - +- CometBroadcastHashJoin (39) - :- CometProject (37) - : +- CometBroadcastHashJoin (36) - : :- CometProject (28) - : : +- CometFilter (27) - : : +- CometWindowExec (26) - : : +- CometFilter (25) - : : +- CometWindowExec (24) - : : +- CometSort (23) - : : +- CometExchange (22) - : : +- CometHashAggregate (21) - : : +- CometExchange (20) - : : +- CometHashAggregate (19) - : : +- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometProject (8) - : : : : +- CometBroadcastHashJoin (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) - : : : : +- CometBroadcastExchange (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometFilter (10) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) - : : +- CometBroadcastExchange (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) - : +- CometBroadcastExchange (35) - : +- CometProject (34) - : +- CometWindowExec (33) - : +- CometSort (32) - : +- CometExchange (31) - : +- CometHashAggregate (30) - : +- ReusedExchange (29) - +- ReusedExchange (38) +TakeOrderedAndProject (48) ++- * Project (47) + +- * BroadcastHashJoin Inner BuildRight (46) + :- * Project (39) + : +- * BroadcastHashJoin Inner BuildRight (38) + : :- * Project (29) + : : +- * Filter (28) + : : +- Window (27) + : : +- * Filter (26) + : : +- Window (25) + : : +- * CometColumnarToRow (24) + : : +- CometSort (23) + : : +- CometExchange (22) + : : +- CometHashAggregate (21) + : : +- CometExchange (20) + : : +- CometHashAggregate (19) + : : +- CometProject (18) + : : +- CometBroadcastHashJoin (17) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometProject (8) + : : : : +- CometBroadcastHashJoin (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (1) + : : : : +- CometBroadcastExchange (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometFilter (10) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) + : : +- CometBroadcastExchange (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center (14) + : +- BroadcastExchange (37) + : +- * Project (36) + : +- Window (35) + : +- * CometColumnarToRow (34) + : +- CometSort (33) + : +- CometExchange (32) + : +- CometHashAggregate (31) + : +- ReusedExchange (30) + +- BroadcastExchange (45) + +- * Project (44) + +- Window (43) + +- * CometColumnarToRow (42) + +- CometSort (41) + +- ReusedExchange (40) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.item @@ -153,107 +159,131 @@ Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#15, 5), ENSURE_REQU Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#15 ASC NULLS FIRST, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(24) CometWindowExec +(24) CometColumnarToRow [codegen id : 1] Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19], [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] -(25) CometFilter +(25) Window +Input [7]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18] +Arguments: [rank(d_year#12, d_moy#13) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#19], [i_category#5, i_brand#4, cc_name#15], [d_year#12 ASC NULLS FIRST, d_moy#13 ASC NULLS FIRST] + +(26) Filter [codegen id : 2] Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] Condition : (isnotnull(d_year#12) AND (d_year#12 = 1999)) -(26) CometWindowExec +(27) Window Input [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20], [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] +Arguments: [avg(_w0#18) windowspecdefinition(i_category#5, i_brand#4, cc_name#15, d_year#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS avg_monthly_sales#20], [i_category#5, i_brand#4, cc_name#15, d_year#12] -(27) CometFilter +(28) Filter [codegen id : 7] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND CASE WHEN (avg_monthly_sales#20 > 0.000000) THEN ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000) END) -(28) CometProject +(29) Project [codegen id : 7] +Output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] Input [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -(29) ReusedExchange [Reuses operator id: 20] +(30) ReusedExchange [Reuses operator id: 20] Output [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] -(30) CometHashAggregate +(31) CometHashAggregate Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum#24] Keys [5]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23] Functions [1]: [sum(UnscaledValue(cs_sales_price#25))] -(31) CometExchange +(32) CometExchange Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: hashpartitioning(i_category#5, i_brand#4, cc_name#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(32) CometSort +(33) CometSort Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#21 ASC NULLS FIRST, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(33) CometWindowExec +(34) CometColumnarToRow [codegen id : 3] +Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] + +(35) Window Input [6]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17] -Arguments: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26], [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] +Arguments: [rank(d_year#22, d_moy#23) windowspecdefinition(i_category#5, i_brand#4, cc_name#21, d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#26], [i_category#5, i_brand#4, cc_name#21], [d_year#22 ASC NULLS FIRST, d_moy#23 ASC NULLS FIRST] -(34) CometProject +(36) Project [codegen id : 4] +Output [5]: [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] Input [7]: [i_category#5, i_brand#4, cc_name#21, d_year#22, d_moy#23, sum_sales#17, rn#26] -Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26], [i_category#5 AS i_category#27, i_brand#4 AS i_brand#28, cc_name#21, sum_sales#17 AS sum_sales#29, rn#26] -(35) CometBroadcastExchange +(37) BroadcastExchange Input [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] + 1)),false), [plan_id=4] -(36) CometBroadcastHashJoin -Left output [8]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19] -Right output [5]: [i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)], Inner, BuildRight +(38) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#27, i_brand#28, cc_name#21, (rn#26 + 1)] +Join type: Inner +Join condition: None -(37) CometProject +(39) Project [codegen id : 7] +Output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] Input [13]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, i_category#27, i_brand#28, cc_name#21, sum_sales#29, rn#26] -Arguments: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29], [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -(38) ReusedExchange [Reuses operator id: 35] -Output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] +(40) ReusedExchange [Reuses operator id: 32] +Output [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] + +(41) CometSort +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17], [i_category#5 ASC NULLS FIRST, i_brand#4 ASC NULLS FIRST, cc_name#30 ASC NULLS FIRST, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] + +(42) CometColumnarToRow [codegen id : 5] +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] + +(43) Window +Input [6]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17] +Arguments: [rank(d_year#31, d_moy#32) windowspecdefinition(i_category#5, i_brand#4, cc_name#30, d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rn#33], [i_category#5, i_brand#4, cc_name#30], [d_year#31 ASC NULLS FIRST, d_moy#32 ASC NULLS FIRST] + +(44) Project [codegen id : 6] +Output [5]: [i_category#5 AS i_category#34, i_brand#4 AS i_brand#35, cc_name#30, sum_sales#17 AS sum_sales#36, rn#33] +Input [7]: [i_category#5, i_brand#4, cc_name#30, d_year#31, d_moy#32, sum_sales#17, rn#33] -(39) CometBroadcastHashJoin -Left output [9]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29] -Right output [5]: [i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -Arguments: [i_category#5, i_brand#4, cc_name#15, rn#19], [i_category#30, i_brand#31, cc_name#32, (rn#34 - 1)], Inner, BuildRight +(45) BroadcastExchange +Input [5]: [i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true], input[1, string, true], input[2, string, true], (input[4, int, false] - 1)),false), [plan_id=5] -(40) CometProject -Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#30, i_brand#31, cc_name#32, sum_sales#33, rn#34] -Arguments: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#35, sum_sales#33 AS nsum#36] +(46) BroadcastHashJoin [codegen id : 7] +Left keys [4]: [i_category#5, i_brand#4, cc_name#15, rn#19] +Right keys [4]: [i_category#34, i_brand#35, cc_name#30, (rn#33 - 1)] +Join type: Inner +Join condition: None -(41) CometTakeOrderedAndProject -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST,d_year#12 ASC NULLS FIRST], output=[i_category#5,i_brand#4,d_year#12,d_moy#13,avg_monthly_sales#20,sum_sales#17,psum#35,nsum#36]), [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36], 100, 0, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +(47) Project [codegen id : 7] +Output [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, sum_sales#29 AS psum#37, sum_sales#36 AS nsum#38] +Input [14]: [i_category#5, i_brand#4, cc_name#15, d_year#12, d_moy#13, sum_sales#17, avg_monthly_sales#20, rn#19, sum_sales#29, i_category#34, i_brand#35, cc_name#30, sum_sales#36, rn#33] -(42) CometColumnarToRow [codegen id : 1] -Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#35, nsum#36] +(48) TakeOrderedAndProject +Input [8]: [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] +Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#12 ASC NULLS FIRST], [i_category#5, i_brand#4, d_year#12, d_moy#13, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38] ===== Subqueries ===== Subquery:1 Hosting operator id = 4 Hosting Expression = cs_sold_date_sk#9 IN dynamicpruning#10 -BroadcastExchange (46) -+- * CometColumnarToRow (45) - +- CometFilter (44) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (43) +BroadcastExchange (52) ++- * CometColumnarToRow (51) + +- CometFilter (50) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (49) -(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [3]: [d_date_sk#11, d_year#12, d_moy#13] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [Or(Or(EqualTo(d_year,1999),And(EqualTo(d_year,1998),EqualTo(d_moy,12))),And(EqualTo(d_year,2000),EqualTo(d_moy,1))), IsNotNull(d_date_sk)] ReadSchema: struct -(44) CometFilter +(50) CometFilter Input [3]: [d_date_sk#11, d_year#12, d_moy#13] Condition : ((((d_year#12 = 1999) OR ((d_year#12 = 1998) AND (d_moy#13 = 12))) OR ((d_year#12 = 2000) AND (d_moy#13 = 1))) AND isnotnull(d_date_sk#11)) -(45) CometColumnarToRow [codegen id : 1] +(51) CometColumnarToRow [codegen id : 1] Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -(46) BroadcastExchange +(52) BroadcastExchange Input [3]: [d_date_sk#11, d_year#12, d_moy#13] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=4] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=6] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt index 00de5f32a1..7bee0e61e8 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/extended.txt @@ -1,76 +1,78 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometFilter - : : +- CometWindowExec - : : +- CometFilter - : : +- CometWindowExec - : : +- CometSort - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometExchange - : : +- CometHashAggregate - : : +- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometBroadcastHashJoin - : : : : :- CometProject - : : : : : +- CometFilter - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : : +- CometBroadcastExchange - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : : +- SubqueryBroadcast - : : : : +- BroadcastExchange - : : : : +- CometColumnarToRow - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - : +- CometBroadcastExchange - : +- CometProject - : +- CometWindowExec - : +- CometSort - : +- CometExchange - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometProject - : : : +- CometBroadcastHashJoin - : : : :- CometProject - : : : : +- CometFilter - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - : : : +- CometBroadcastExchange - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center - +- CometBroadcastExchange - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- BroadcastHashJoin + :- Project + : +- BroadcastHashJoin + : :- Project + : : +- Filter + : : +- Window + : : +- Filter + : : +- Window [COMET: Window expressions are not supported] + : : +- CometColumnarToRow + : : +- CometSort + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometExchange + : : +- CometHashAggregate + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometBroadcastHashJoin + : : : : :- CometProject + : : : : : +- CometFilter + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : : +- CometBroadcastExchange + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : : +- SubqueryBroadcast + : : : : +- BroadcastExchange + : : : : +- CometColumnarToRow + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + : +- BroadcastExchange + : +- Project + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometExchange + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometProject + : : : +- CometBroadcastHashJoin + : : : :- CometProject + : : : : +- CometFilter + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + : : : +- CometBroadcastExchange + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center + +- BroadcastExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -100,4 +102,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center -Comet accelerated 91 out of 97 eligible operators (93%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 75 out of 97 eligible operators (77%). Final plan contains 6 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt index 7a51065baf..d655789fe6 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/simplified.txt @@ -1,51 +1,71 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometProject [sum_sales,sum_sales] [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,psum,nsum] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] - CometBroadcastHashJoin [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,i_category,i_brand,cc_name,sum_sales,rn] - CometProject [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn,avg_monthly_sales] - CometFilter [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] - CometExchange [i_category,i_brand,cc_name] #1 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] - CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 - CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] - CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] - CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] - CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] - CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] - CometFilter [i_item_sk,i_brand,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] - CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 - CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 - CometFilter [d_date_sk,d_year,d_moy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] - CometBroadcastExchange [cc_call_center_sk,cc_name] #6 - CometFilter [cc_call_center_sk,cc_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] - CometBroadcastExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 - CometProject [i_category,i_brand,sum_sales] [i_category,i_brand,cc_name,sum_sales,rn] - CometWindowExec [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,rn] - CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] - CometExchange [i_category,i_brand,cc_name] #8 - CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] - ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 - ReusedExchange [i_category,i_brand,cc_name,sum_sales,rn] #7 +TakeOrderedAndProject [sum_sales,avg_monthly_sales,d_year,i_category,i_brand,d_moy,psum,nsum] + WholeStageCodegen (7) + Project [i_category,i_brand,d_year,d_moy,avg_monthly_sales,sum_sales,sum_sales,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn,sum_sales] + BroadcastHashJoin [i_category,i_brand,cc_name,rn,i_category,i_brand,cc_name,rn] + Project [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,avg_monthly_sales,rn] + Filter [avg_monthly_sales,sum_sales] + InputAdapter + Window [_w0,i_category,i_brand,cc_name,d_year] + WholeStageCodegen (2) + Filter [d_year] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0] + CometExchange [i_category,i_brand,cc_name] #1 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,_w0,sum(UnscaledValue(cs_sales_price))] + CometExchange [i_category,i_brand,cc_name,d_year,d_moy] #2 + CometHashAggregate [cs_sales_price] [i_category,i_brand,cc_name,d_year,d_moy,sum] + CometProject [i_brand,i_category,cs_sales_price,d_year,d_moy,cc_name] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy,cc_call_center_sk,cc_name] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,d_year,d_moy] + CometBroadcastHashJoin [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk,d_date_sk,d_year,d_moy] + CometProject [i_brand,i_category,cs_call_center_sk,cs_sales_price,cs_sold_date_sk] + CometBroadcastHashJoin [i_item_sk,i_brand,i_category,cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometProject [i_brand,i_category] [i_item_sk,i_brand,i_category] + CometFilter [i_item_sk,i_brand,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_category] + CometBroadcastExchange [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] #3 + CometFilter [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_call_center_sk,cs_item_sk,cs_sales_price,cs_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [d_date_sk,d_year,d_moy] #5 + CometFilter [d_date_sk,d_year,d_moy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_year,d_moy] + CometBroadcastExchange [cc_call_center_sk,cc_name] #6 + CometFilter [cc_call_center_sk,cc_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.call_center [cc_call_center_sk,cc_name] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (4) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (3) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + CometExchange [i_category,i_brand,cc_name] #8 + CometHashAggregate [sum] [i_category,i_brand,cc_name,d_year,d_moy,sum_sales,sum(UnscaledValue(cs_sales_price))] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum] #2 + InputAdapter + BroadcastExchange #9 + WholeStageCodegen (6) + Project [i_category,i_brand,cc_name,sum_sales,rn] + InputAdapter + Window [d_year,d_moy,i_category,i_brand,cc_name] + WholeStageCodegen (5) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] + ReusedExchange [i_category,i_brand,cc_name,d_year,d_moy,sum_sales] #8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt index 26a54ba072..ac04f86f96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (70) -+- CometTakeOrderedAndProject (69) - +- CometFilter (68) - +- CometWindowExec (67) +TakeOrderedAndProject (70) ++- * Filter (69) + +- Window (68) + +- * CometColumnarToRow (67) +- CometSort (66) +- CometExchange (65) +- CometUnion (64) @@ -376,20 +376,20 @@ Arguments: hashpartitioning(i_category#99, 5), ENSURE_REQUIREMENTS, CometNativeS Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] Arguments: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107], [i_category#99 ASC NULLS FIRST, sumsales#107 DESC NULLS LAST] -(67) CometWindowExec +(67) CometColumnarToRow [codegen id : 1] Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Arguments: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152], [rank(sumsales#107) windowspecdefinition(i_category#99, sumsales#107 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#152], [i_category#99], [sumsales#107 DESC NULLS LAST] -(68) CometFilter -Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] -Condition : (rk#152 <= 100) +(68) Window +Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] +Arguments: [rank(sumsales#107) windowspecdefinition(i_category#99, sumsales#107 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#152], [i_category#99], [sumsales#107 DESC NULLS LAST] -(69) CometTakeOrderedAndProject +(69) Filter [codegen id : 2] Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#99 ASC NULLS FIRST,i_class#100 ASC NULLS FIRST,i_brand#101 ASC NULLS FIRST,i_product_name#102 ASC NULLS FIRST,d_year#103 ASC NULLS FIRST,d_qoy#104 ASC NULLS FIRST,d_moy#105 ASC NULLS FIRST,s_store_id#106 ASC NULLS FIRST,sumsales#107 ASC NULLS FIRST,rk#152 ASC NULLS FIRST], output=[i_category#99,i_class#100,i_brand#101,i_product_name#102,d_year#103,d_qoy#104,d_moy#105,s_store_id#106,sumsales#107,rk#152]), [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152], 100, 0, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_brand#101 ASC NULLS FIRST, i_product_name#102 ASC NULLS FIRST, d_year#103 ASC NULLS FIRST, d_qoy#104 ASC NULLS FIRST, d_moy#105 ASC NULLS FIRST, s_store_id#106 ASC NULLS FIRST, sumsales#107 ASC NULLS FIRST, rk#152 ASC NULLS FIRST], [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] +Condition : (rk#152 <= 100) -(70) CometColumnarToRow [codegen id : 1] +(70) TakeOrderedAndProject Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] +Arguments: 100, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_brand#101 ASC NULLS FIRST, i_product_name#102 ASC NULLS FIRST, d_year#103 ASC NULLS FIRST, d_qoy#104 ASC NULLS FIRST, d_moy#105 ASC NULLS FIRST, s_store_id#106 ASC NULLS FIRST, sumsales#107 ASC NULLS FIRST, rk#152 ASC NULLS FIRST], [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt index 2eaf297aaa..e1884afb56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometUnion @@ -291,4 +291,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 264 out of 282 eligible operators (93%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt index cd140526f4..5ca258aeca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.native_iceberg_compat/simplified.txt @@ -1,80 +1,82 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometFilter [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometWindowExec [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + WholeStageCodegen (2) + Filter [rk] + InputAdapter + Window [sumsales,i_category] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometExchange [i_category] #1 + CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] + CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] + CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] + CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name] #10 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] + CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand] #11 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] + CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class] #12 + CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] + CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category] #13 + CometHashAggregate [sumsales] [i_category,sum,isEmpty] + CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange #14 + CometHashAggregate [sumsales] [sum,isEmpty] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index 26a54ba072..ac04f86f96 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (70) -+- CometTakeOrderedAndProject (69) - +- CometFilter (68) - +- CometWindowExec (67) +TakeOrderedAndProject (70) ++- * Filter (69) + +- Window (68) + +- * CometColumnarToRow (67) +- CometSort (66) +- CometExchange (65) +- CometUnion (64) @@ -376,20 +376,20 @@ Arguments: hashpartitioning(i_category#99, 5), ENSURE_REQUIREMENTS, CometNativeS Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] Arguments: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107], [i_category#99 ASC NULLS FIRST, sumsales#107 DESC NULLS LAST] -(67) CometWindowExec +(67) CometColumnarToRow [codegen id : 1] Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] -Arguments: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152], [rank(sumsales#107) windowspecdefinition(i_category#99, sumsales#107 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#152], [i_category#99], [sumsales#107 DESC NULLS LAST] -(68) CometFilter -Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] -Condition : (rk#152 <= 100) +(68) Window +Input [9]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107] +Arguments: [rank(sumsales#107) windowspecdefinition(i_category#99, sumsales#107 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#152], [i_category#99], [sumsales#107 DESC NULLS LAST] -(69) CometTakeOrderedAndProject +(69) Filter [codegen id : 2] Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[i_category#99 ASC NULLS FIRST,i_class#100 ASC NULLS FIRST,i_brand#101 ASC NULLS FIRST,i_product_name#102 ASC NULLS FIRST,d_year#103 ASC NULLS FIRST,d_qoy#104 ASC NULLS FIRST,d_moy#105 ASC NULLS FIRST,s_store_id#106 ASC NULLS FIRST,sumsales#107 ASC NULLS FIRST,rk#152 ASC NULLS FIRST], output=[i_category#99,i_class#100,i_brand#101,i_product_name#102,d_year#103,d_qoy#104,d_moy#105,s_store_id#106,sumsales#107,rk#152]), [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152], 100, 0, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_brand#101 ASC NULLS FIRST, i_product_name#102 ASC NULLS FIRST, d_year#103 ASC NULLS FIRST, d_qoy#104 ASC NULLS FIRST, d_moy#105 ASC NULLS FIRST, s_store_id#106 ASC NULLS FIRST, sumsales#107 ASC NULLS FIRST, rk#152 ASC NULLS FIRST], [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] +Condition : (rk#152 <= 100) -(70) CometColumnarToRow [codegen id : 1] +(70) TakeOrderedAndProject Input [10]: [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] +Arguments: 100, [i_category#99 ASC NULLS FIRST, i_class#100 ASC NULLS FIRST, i_brand#101 ASC NULLS FIRST, i_product_name#102 ASC NULLS FIRST, d_year#103 ASC NULLS FIRST, d_qoy#104 ASC NULLS FIRST, d_moy#105 ASC NULLS FIRST, s_store_id#106 ASC NULLS FIRST, sumsales#107 ASC NULLS FIRST, rk#152 ASC NULLS FIRST], [i_category#99, i_class#100, i_brand#101, i_product_name#102, d_year#103, d_qoy#104, d_moy#105, s_store_id#106, sumsales#107, rk#152] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt index 2eaf297aaa..e1884afb56 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometFilter - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometUnion @@ -291,4 +291,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 264 out of 282 eligible operators (93%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 261 out of 282 eligible operators (92%). Final plan contains 10 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index cd140526f4..5ca258aeca 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -1,80 +1,82 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometFilter [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometWindowExec [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] - CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometExchange [i_category] #1 - CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 - CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] - CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] - CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] - CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] - CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #3 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 - CometProject [d_date_sk,d_year,d_moy,d_qoy] - CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] - CometBroadcastExchange [s_store_sk,s_store_id] #5 - CometProject [s_store_id] [s_store_sk,s_store_id] - CometFilter [s_store_sk,s_store_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] - CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 - CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] - CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] - CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] - CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand,i_product_name] #10 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] - CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class,i_brand] #11 - CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] - CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category,i_class] #12 - CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] - CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange [i_category] #13 - CometHashAggregate [sumsales] [i_category,sum,isEmpty] - CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 - CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] - CometExchange #14 - CometHashAggregate [sumsales] [sum,isEmpty] - CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] - ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 +TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,rk] + WholeStageCodegen (2) + Filter [rk] + InputAdapter + Window [sumsales,i_category] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometExchange [i_category] #1 + CometUnion [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 + CometHashAggregate [ss_sales_price,ss_quantity] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] + CometProject [ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_brand,i_class,i_category,i_product_name] + CometBroadcastHashJoin [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id,i_item_sk,i_brand,i_class,i_category,i_product_name] + CometProject [ss_item_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,d_year,d_moy,d_qoy] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk,d_date_sk,d_year,d_moy,d_qoy] + CometFilter [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_store_sk,ss_quantity,ss_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #3 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [d_date_sk,d_year,d_moy,d_qoy] #4 + CometProject [d_date_sk,d_year,d_moy,d_qoy] + CometFilter [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq,d_year,d_moy,d_qoy] + CometBroadcastExchange [s_store_sk,s_store_id] #5 + CometProject [s_store_id] [s_store_sk,s_store_id] + CometFilter [s_store_sk,s_store_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_store_id] + CometBroadcastExchange [i_item_sk,i_brand,i_class,i_category,i_product_name] #6 + CometProject [i_brand,i_class,i_category,i_product_name] [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometFilter [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_brand,i_class,i_category,i_product_name] + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sum,isEmpty] + CometHashAggregate [s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sum,isEmpty] + CometHashAggregate [d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name,d_year] #9 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,d_year,sum,isEmpty] + CometHashAggregate [d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand,i_product_name] #10 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,i_product_name,sum,isEmpty] + CometHashAggregate [d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,i_product_name,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class,i_brand] #11 + CometHashAggregate [sumsales] [i_category,i_class,i_brand,sum,isEmpty] + CometHashAggregate [i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,i_brand,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category,i_class] #12 + CometHashAggregate [sumsales] [i_category,i_class,sum,isEmpty] + CometHashAggregate [i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,i_class,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange [i_category] #13 + CometHashAggregate [sumsales] [i_category,sum,isEmpty] + CometHashAggregate [i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [i_category,sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty] [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum(sumsales)] + CometExchange #14 + CometHashAggregate [sumsales] [sum,isEmpty] + CometHashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sumsales,sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00))] + ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt index ae6e2e9bd2..02cdbb3c66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/explain.txt @@ -1,61 +1,69 @@ == Physical Plan == -* CometColumnarToRow (57) -+- CometTakeOrderedAndProject (56) - +- CometProject (55) - +- CometWindowExec (54) - +- CometSort (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometUnion (48) - :- CometHashAggregate (37) - : +- CometExchange (36) - : +- CometHashAggregate (35) - : +- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometWindowExec (26) - : +- CometSort (25) - : +- CometHashAggregate (24) - : +- CometExchange (23) - : +- CometHashAggregate (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (12) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) - : : +- CometBroadcastExchange (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : +- ReusedExchange (19) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometHashAggregate (39) - : +- ReusedExchange (38) - +- CometHashAggregate (47) - +- CometExchange (46) - +- CometHashAggregate (45) - +- CometHashAggregate (44) - +- ReusedExchange (43) +TakeOrderedAndProject (65) ++- * Project (64) + +- Window (63) + +- * CometColumnarToRow (62) + +- CometSort (61) + +- CometExchange (60) + +- CometHashAggregate (59) + +- CometColumnarExchange (58) + +- * HashAggregate (57) + +- Union (56) + :- * HashAggregate (41) + : +- * CometColumnarToRow (40) + : +- CometColumnarExchange (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * CometColumnarToRow (9) + : : +- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * BroadcastHashJoin LeftSemi BuildRight (33) + : :- * CometColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) + : +- BroadcastExchange (32) + : +- * Project (31) + : +- * Filter (30) + : +- Window (29) + : +- * CometColumnarToRow (28) + : +- CometSort (27) + : +- CometHashAggregate (26) + : +- CometExchange (25) + : +- CometHashAggregate (24) + : +- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- CometProject (20) + : : +- CometBroadcastHashJoin (19) + : : :- CometFilter (14) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) + : : +- CometBroadcastExchange (18) + : : +- CometProject (17) + : : +- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) + : +- ReusedExchange (21) + :- * HashAggregate (48) + : +- * CometColumnarToRow (47) + : +- CometColumnarExchange (46) + : +- * HashAggregate (45) + : +- * HashAggregate (44) + : +- * CometColumnarToRow (43) + : +- ReusedExchange (42) + +- * HashAggregate (55) + +- * CometColumnarToRow (54) + +- CometColumnarExchange (53) + +- * HashAggregate (52) + +- * HashAggregate (51) + +- * CometColumnarToRow (50) + +- ReusedExchange (49) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -98,18 +106,24 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(9) CometColumnarToRow [codegen id : 4] +Input [2]: [ss_store_sk#1, ss_net_profit#2] + +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(10) CometFilter +(11) CometFilter Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(12) CometColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] @@ -117,238 +131,273 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(12) CometFilter +(14) CometFilter Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Condition : isnotnull(ss_store_sk#10) -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter +(16) CometFilter Input [2]: [s_store_sk#14, s_state#15] Condition : isnotnull(s_store_sk#14) -(15) CometProject +(17) CometProject Input [2]: [s_store_sk#14, s_state#15] Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] -(16) CometBroadcastExchange +(18) CometBroadcastExchange Input [2]: [s_store_sk#14, s_state#16] Arguments: [s_store_sk#14, s_state#16] -(17) CometBroadcastHashJoin +(19) CometBroadcastHashJoin Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Right output [2]: [s_store_sk#14, s_state#16] Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight -(18) CometProject +(20) CometProject Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -(19) ReusedExchange [Reuses operator id: 6] +(21) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#17] -(20) CometBroadcastHashJoin +(22) CometBroadcastHashJoin Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] Right output [1]: [d_date_sk#17] Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight -(21) CometProject +(23) CometProject Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] -(22) CometHashAggregate +(24) CometHashAggregate Input [2]: [ss_net_profit#11, s_state#16] Keys [1]: [s_state#16] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] -(23) CometExchange +(25) CometExchange Input [2]: [s_state#16, sum#18] Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(24) CometHashAggregate +(26) CometHashAggregate Input [2]: [s_state#16, sum#18] Keys [1]: [s_state#16] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -(25) CometSort +(27) CometSort Input [3]: [s_state#16, _w0#19, s_state#16] Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(26) CometWindowExec +(28) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16, ranking#20], [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] -(27) CometFilter +(29) Window +Input [3]: [s_state#16, _w0#19, s_state#16] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] + +(30) Filter [codegen id : 2] Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] Condition : (ranking#20 <= 5) -(28) CometProject +(31) Project [codegen id : 2] +Output [1]: [s_state#16] Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Arguments: [s_state#16], [s_state#16] -(29) CometBroadcastExchange +(32) BroadcastExchange Input [1]: [s_state#16] -Arguments: [s_state#16] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(30) CometBroadcastHashJoin -Left output [3]: [s_store_sk#7, s_county#8, s_state#9] -Right output [1]: [s_state#16] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)], [s_state#16], LeftSemi, BuildRight +(33) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] +Right keys [1]: [s_state#16] +Join type: LeftSemi +Join condition: None -(31) CometProject +(34) Project [codegen id : 3] +Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Arguments: [s_store_sk#7, s_county#8, s_state#21], [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -(32) CometBroadcastExchange +(35) BroadcastExchange Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: [s_store_sk#7, s_county#8, s_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) CometBroadcastHashJoin -Left output [2]: [ss_store_sk#1, ss_net_profit#2] -Right output [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: [ss_store_sk#1], [s_store_sk#7], Inner, BuildRight +(36) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#7] +Join type: Inner +Join condition: None -(34) CometProject +(37) Project [codegen id : 4] +Output [3]: [ss_net_profit#2, s_county#8, s_state#21] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] -Arguments: [ss_net_profit#2, s_county#8, s_state#21], [ss_net_profit#2, s_county#8, s_state#21] -(35) CometHashAggregate +(38) HashAggregate [codegen id : 4] Input [3]: [ss_net_profit#2, s_county#8, s_state#21] Keys [2]: [s_state#21, s_county#8] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum#22] +Results [3]: [s_state#21, s_county#8, sum#23] -(36) CometExchange -Input [3]: [s_state#21, s_county#8, sum#22] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(39) CometColumnarExchange +Input [3]: [s_state#21, s_county#8, sum#23] +Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(37) CometHashAggregate -Input [3]: [s_state#21, s_county#8, sum#22] +(40) CometColumnarToRow [codegen id : 5] +Input [3]: [s_state#21, s_county#8, sum#23] + +(41) HashAggregate [codegen id : 5] +Input [3]: [s_state#21, s_county#8, sum#23] Keys [2]: [s_state#21, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] + +(42) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_state#21, s_county#31, sum#32] -(38) ReusedExchange [Reuses operator id: 36] -Output [3]: [s_state#21, s_county#23, sum#24] +(43) CometColumnarToRow [codegen id : 10] +Input [3]: [s_state#21, s_county#31, sum#32] -(39) CometHashAggregate -Input [3]: [s_state#21, s_county#23, sum#24] -Keys [2]: [s_state#21, s_county#23] -Functions [1]: [sum(UnscaledValue(ss_net_profit#25))] +(44) HashAggregate [codegen id : 10] +Input [3]: [s_state#21, s_county#31, sum#32] +Keys [2]: [s_state#21, s_county#31] +Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] -(40) CometHashAggregate -Input [2]: [total_sum#26, s_state#21] +(45) HashAggregate [codegen id : 10] +Input [2]: [total_sum#34, s_state#21] Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#26)] +Functions [1]: [partial_sum(total_sum#34)] +Aggregate Attributes [2]: [sum#35, isEmpty#36] +Results [3]: [s_state#21, sum#37, isEmpty#38] -(41) CometExchange -Input [3]: [s_state#21, sum#27, isEmpty#28] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(46) CometColumnarExchange +Input [3]: [s_state#21, sum#37, isEmpty#38] +Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(42) CometHashAggregate -Input [3]: [s_state#21, sum#27, isEmpty#28] +(47) CometColumnarToRow [codegen id : 11] +Input [3]: [s_state#21, sum#37, isEmpty#38] + +(48) HashAggregate [codegen id : 11] +Input [3]: [s_state#21, sum#37, isEmpty#38] Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#26)] +Functions [1]: [sum(total_sum#34)] +Aggregate Attributes [1]: [sum(total_sum#34)#39] +Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] + +(49) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_state#21, s_county#45, sum#46] -(43) ReusedExchange [Reuses operator id: 36] -Output [3]: [s_state#21, s_county#29, sum#30] +(50) CometColumnarToRow [codegen id : 16] +Input [3]: [s_state#21, s_county#45, sum#46] -(44) CometHashAggregate -Input [3]: [s_state#21, s_county#29, sum#30] -Keys [2]: [s_state#21, s_county#29] -Functions [1]: [sum(UnscaledValue(ss_net_profit#31))] +(51) HashAggregate [codegen id : 16] +Input [3]: [s_state#21, s_county#45, sum#46] +Keys [2]: [s_state#21, s_county#45] +Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#34] -(45) CometHashAggregate -Input [1]: [total_sum#26] +(52) HashAggregate [codegen id : 16] +Input [1]: [total_sum#34] Keys: [] -Functions [1]: [partial_sum(total_sum#26)] +Functions [1]: [partial_sum(total_sum#34)] +Aggregate Attributes [2]: [sum#48, isEmpty#49] +Results [2]: [sum#50, isEmpty#51] + +(53) CometColumnarExchange +Input [2]: [sum#50, isEmpty#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(46) CometExchange -Input [2]: [sum#32, isEmpty#33] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(54) CometColumnarToRow [codegen id : 17] +Input [2]: [sum#50, isEmpty#51] -(47) CometHashAggregate -Input [2]: [sum#32, isEmpty#33] +(55) HashAggregate [codegen id : 17] +Input [2]: [sum#50, isEmpty#51] Keys: [] -Functions [1]: [sum(total_sum#26)] +Functions [1]: [sum(total_sum#34)] +Aggregate Attributes [1]: [sum(total_sum#34)#52] +Results [6]: [sum(total_sum#34)#52 AS total_sum#53, null AS s_state#54, null AS s_county#55, 1 AS g_state#56, 1 AS g_county#57, 2 AS lochierarchy#58] -(48) CometUnion -Child 0 Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] -Child 1 Input [6]: [total_sum#40, s_state#21, s_county#41, g_state#42, g_county#43, lochierarchy#44] -Child 2 Input [6]: [total_sum#45, s_state#46, s_county#47, g_state#48, g_county#49, lochierarchy#50] +(56) Union -(49) CometHashAggregate -Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] -Keys [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +(57) HashAggregate [codegen id : 18] +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Functions: [] +Aggregate Attributes: [] +Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -(50) CometExchange -Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] -Arguments: hashpartitioning(total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(58) CometColumnarExchange +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(51) CometHashAggregate -Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] -Keys [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +(59) CometHashAggregate +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Functions: [] -(52) CometExchange -Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] -Arguments: hashpartitioning(lochierarchy#39, _w0#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(60) CometExchange +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] +Arguments: hashpartitioning(lochierarchy#30, _w0#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(53) CometSort -Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] -Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51], [lochierarchy#39 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#34 DESC NULLS LAST] +(61) CometSort +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] +Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59], [lochierarchy#30 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] -(54) CometWindowExec -Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] -Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51, rank_within_parent#52], [rank(total_sum#34) windowspecdefinition(lochierarchy#39, _w0#51, total_sum#34 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#39, _w0#51], [total_sum#34 DESC NULLS LAST] +(62) CometColumnarToRow [codegen id : 19] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -(55) CometProject -Input [6]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51, rank_within_parent#52] -Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52], [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] +(63) Window +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] +Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#59, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#30, _w0#59], [total_sum#25 DESC NULLS LAST] -(56) CometTakeOrderedAndProject -Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#39 DESC NULLS LAST,CASE WHEN (lochierarchy#39 = 0) THEN s_state#35 END ASC NULLS FIRST,rank_within_parent#52 ASC NULLS FIRST], output=[total_sum#34,s_state#35,s_county#36,lochierarchy#39,rank_within_parent#52]), [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52], 100, 0, [lochierarchy#39 DESC NULLS LAST, CASE WHEN (lochierarchy#39 = 0) THEN s_state#35 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] +(64) Project [codegen id : 20] +Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] +Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59, rank_within_parent#60] -(57) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] +(65) TakeOrderedAndProject +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] +Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) +BroadcastExchange (70) ++- * CometColumnarToRow (69) + +- CometProject (68) + +- CometFilter (67) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(59) CometFilter +(67) CometFilter Input [2]: [d_date_sk#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(60) CometProject +(68) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(61) CometColumnarToRow [codegen id : 1] +(69) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(62) BroadcastExchange +(70) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt index 5b14945482..a187c52d0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/extended.txt @@ -1,159 +1,173 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometHashAggregate - : +- CometExchange + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 144 out of 153 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 102 out of 153 eligible operators (66%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt index 91be7bc024..8de8b7e637 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a.native_iceberg_compat/simplified.txt @@ -1,68 +1,100 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,rank_within_parent] - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - CometHashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - CometUnion [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - CometHashAggregate [sum] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(UnscaledValue(ss_net_profit)),s_state,s_county] - CometExchange [s_state,s_county] #3 - CometHashAggregate [ss_net_profit] [s_state,s_county,sum] - CometProject [ss_net_profit,s_county,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_county,s_state] - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (20) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + WholeStageCodegen (18) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] + CometExchange [s_state] #8 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #9 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (11) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #10 + WholeStageCodegen (10) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #11 + WholeStageCodegen (16) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [s_store_sk,s_county,s_state] #6 - CometProject [s_state] [s_store_sk,s_county,s_state] - CometBroadcastHashJoin [s_store_sk,s_county,s_state,s_state] - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - CometBroadcastExchange [s_state] #7 - CometProject [s_state] - CometFilter [s_state,_w0,ranking] - CometWindowExec [s_state,_w0,ranking] - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - CometHashAggregate [sum,isEmpty] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(total_sum)] - CometExchange [s_state] #10 - CometHashAggregate [total_sum] [s_state,sum,isEmpty] - CometHashAggregate [s_county,sum] [total_sum,s_state,sum(UnscaledValue(ss_net_profit))] - ReusedExchange [s_state,s_county,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(total_sum)] - CometExchange #11 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [s_state,s_county,sum] [total_sum,sum(UnscaledValue(ss_net_profit))] - ReusedExchange [s_state,s_county,sum] #3 + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt index ae6e2e9bd2..02cdbb3c66 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/explain.txt @@ -1,61 +1,69 @@ == Physical Plan == -* CometColumnarToRow (57) -+- CometTakeOrderedAndProject (56) - +- CometProject (55) - +- CometWindowExec (54) - +- CometSort (53) - +- CometExchange (52) - +- CometHashAggregate (51) - +- CometExchange (50) - +- CometHashAggregate (49) - +- CometUnion (48) - :- CometHashAggregate (37) - : +- CometExchange (36) - : +- CometHashAggregate (35) - : +- CometProject (34) - : +- CometBroadcastHashJoin (33) - : :- CometProject (8) - : : +- CometBroadcastHashJoin (7) - : : :- CometFilter (2) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : +- CometBroadcastExchange (6) - : : +- CometProject (5) - : : +- CometFilter (4) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) - : +- CometBroadcastExchange (32) - : +- CometProject (31) - : +- CometBroadcastHashJoin (30) - : :- CometFilter (10) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (9) - : +- CometBroadcastExchange (29) - : +- CometProject (28) - : +- CometFilter (27) - : +- CometWindowExec (26) - : +- CometSort (25) - : +- CometHashAggregate (24) - : +- CometExchange (23) - : +- CometHashAggregate (22) - : +- CometProject (21) - : +- CometBroadcastHashJoin (20) - : :- CometProject (18) - : : +- CometBroadcastHashJoin (17) - : : :- CometFilter (12) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (11) - : : +- CometBroadcastExchange (16) - : : +- CometProject (15) - : : +- CometFilter (14) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (13) - : +- ReusedExchange (19) - :- CometHashAggregate (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometHashAggregate (39) - : +- ReusedExchange (38) - +- CometHashAggregate (47) - +- CometExchange (46) - +- CometHashAggregate (45) - +- CometHashAggregate (44) - +- ReusedExchange (43) +TakeOrderedAndProject (65) ++- * Project (64) + +- Window (63) + +- * CometColumnarToRow (62) + +- CometSort (61) + +- CometExchange (60) + +- CometHashAggregate (59) + +- CometColumnarExchange (58) + +- * HashAggregate (57) + +- Union (56) + :- * HashAggregate (41) + : +- * CometColumnarToRow (40) + : +- CometColumnarExchange (39) + : +- * HashAggregate (38) + : +- * Project (37) + : +- * BroadcastHashJoin Inner BuildRight (36) + : :- * CometColumnarToRow (9) + : : +- CometProject (8) + : : +- CometBroadcastHashJoin (7) + : : :- CometFilter (2) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : +- CometBroadcastExchange (6) + : : +- CometProject (5) + : : +- CometFilter (4) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (3) + : +- BroadcastExchange (35) + : +- * Project (34) + : +- * BroadcastHashJoin LeftSemi BuildRight (33) + : :- * CometColumnarToRow (12) + : : +- CometFilter (11) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (10) + : +- BroadcastExchange (32) + : +- * Project (31) + : +- * Filter (30) + : +- Window (29) + : +- * CometColumnarToRow (28) + : +- CometSort (27) + : +- CometHashAggregate (26) + : +- CometExchange (25) + : +- CometHashAggregate (24) + : +- CometProject (23) + : +- CometBroadcastHashJoin (22) + : :- CometProject (20) + : : +- CometBroadcastHashJoin (19) + : : :- CometFilter (14) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (13) + : : +- CometBroadcastExchange (18) + : : +- CometProject (17) + : : +- CometFilter (16) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (15) + : +- ReusedExchange (21) + :- * HashAggregate (48) + : +- * CometColumnarToRow (47) + : +- CometColumnarExchange (46) + : +- * HashAggregate (45) + : +- * HashAggregate (44) + : +- * CometColumnarToRow (43) + : +- ReusedExchange (42) + +- * HashAggregate (55) + +- * CometColumnarToRow (54) + +- CometColumnarExchange (53) + +- * HashAggregate (52) + +- * HashAggregate (51) + +- * CometColumnarToRow (50) + +- ReusedExchange (49) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -98,18 +106,24 @@ Arguments: [ss_sold_date_sk#3], [d_date_sk#5], Inner, BuildRight Input [4]: [ss_store_sk#1, ss_net_profit#2, ss_sold_date_sk#3, d_date_sk#5] Arguments: [ss_store_sk#1, ss_net_profit#2], [ss_store_sk#1, ss_net_profit#2] -(9) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(9) CometColumnarToRow [codegen id : 4] +Input [2]: [ss_store_sk#1, ss_net_profit#2] + +(10) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [3]: [s_store_sk#7, s_county#8, s_state#9] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(10) CometFilter +(11) CometFilter Input [3]: [s_store_sk#7, s_county#8, s_state#9] Condition : isnotnull(s_store_sk#7) -(11) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales +(12) CometColumnarToRow [codegen id : 3] +Input [3]: [s_store_sk#7, s_county#8, s_state#9] + +(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales Output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Batched: true Location: InMemoryFileIndex [] @@ -117,238 +131,273 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#12), dynamicpruningexpression(ss_so PushedFilters: [IsNotNull(ss_store_sk)] ReadSchema: struct -(12) CometFilter +(14) CometFilter Input [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Condition : isnotnull(ss_store_sk#10) -(13) CometScan [native_iceberg_compat] parquet spark_catalog.default.store +(15) CometScan [native_iceberg_compat] parquet spark_catalog.default.store Output [2]: [s_store_sk#14, s_state#15] Batched: true Location [not included in comparison]/{warehouse_dir}/store] PushedFilters: [IsNotNull(s_store_sk)] ReadSchema: struct -(14) CometFilter +(16) CometFilter Input [2]: [s_store_sk#14, s_state#15] Condition : isnotnull(s_store_sk#14) -(15) CometProject +(17) CometProject Input [2]: [s_store_sk#14, s_state#15] Arguments: [s_store_sk#14, s_state#16], [s_store_sk#14, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#15, 2, true, false, true) AS s_state#16] -(16) CometBroadcastExchange +(18) CometBroadcastExchange Input [2]: [s_store_sk#14, s_state#16] Arguments: [s_store_sk#14, s_state#16] -(17) CometBroadcastHashJoin +(19) CometBroadcastHashJoin Left output [3]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12] Right output [2]: [s_store_sk#14, s_state#16] Arguments: [ss_store_sk#10], [s_store_sk#14], Inner, BuildRight -(18) CometProject +(20) CometProject Input [5]: [ss_store_sk#10, ss_net_profit#11, ss_sold_date_sk#12, s_store_sk#14, s_state#16] Arguments: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16], [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] -(19) ReusedExchange [Reuses operator id: 6] +(21) ReusedExchange [Reuses operator id: 6] Output [1]: [d_date_sk#17] -(20) CometBroadcastHashJoin +(22) CometBroadcastHashJoin Left output [3]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16] Right output [1]: [d_date_sk#17] Arguments: [ss_sold_date_sk#12], [d_date_sk#17], Inner, BuildRight -(21) CometProject +(23) CometProject Input [4]: [ss_net_profit#11, ss_sold_date_sk#12, s_state#16, d_date_sk#17] Arguments: [ss_net_profit#11, s_state#16], [ss_net_profit#11, s_state#16] -(22) CometHashAggregate +(24) CometHashAggregate Input [2]: [ss_net_profit#11, s_state#16] Keys [1]: [s_state#16] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#11))] -(23) CometExchange +(25) CometExchange Input [2]: [s_state#16, sum#18] Arguments: hashpartitioning(s_state#16, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(24) CometHashAggregate +(26) CometHashAggregate Input [2]: [s_state#16, sum#18] Keys [1]: [s_state#16] Functions [1]: [sum(UnscaledValue(ss_net_profit#11))] -(25) CometSort +(27) CometSort Input [3]: [s_state#16, _w0#19, s_state#16] Arguments: [s_state#16, _w0#19, s_state#16], [s_state#16 ASC NULLS FIRST, _w0#19 DESC NULLS LAST] -(26) CometWindowExec +(28) CometColumnarToRow [codegen id : 1] Input [3]: [s_state#16, _w0#19, s_state#16] -Arguments: [s_state#16, _w0#19, s_state#16, ranking#20], [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] -(27) CometFilter +(29) Window +Input [3]: [s_state#16, _w0#19, s_state#16] +Arguments: [rank(_w0#19) windowspecdefinition(s_state#16, _w0#19 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS ranking#20], [s_state#16], [_w0#19 DESC NULLS LAST] + +(30) Filter [codegen id : 2] Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] Condition : (ranking#20 <= 5) -(28) CometProject +(31) Project [codegen id : 2] +Output [1]: [s_state#16] Input [4]: [s_state#16, _w0#19, s_state#16, ranking#20] -Arguments: [s_state#16], [s_state#16] -(29) CometBroadcastExchange +(32) BroadcastExchange Input [1]: [s_state#16] -Arguments: [s_state#16] +Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [plan_id=2] -(30) CometBroadcastHashJoin -Left output [3]: [s_store_sk#7, s_county#8, s_state#9] -Right output [1]: [s_state#16] -Arguments: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)], [s_state#16], LeftSemi, BuildRight +(33) BroadcastHashJoin [codegen id : 3] +Left keys [1]: [staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true)] +Right keys [1]: [s_state#16] +Join type: LeftSemi +Join condition: None -(31) CometProject +(34) Project [codegen id : 3] +Output [3]: [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] Input [3]: [s_store_sk#7, s_county#8, s_state#9] -Arguments: [s_store_sk#7, s_county#8, s_state#21], [s_store_sk#7, s_county#8, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, s_state#9, 2, true, false, true) AS s_state#21] -(32) CometBroadcastExchange +(35) BroadcastExchange Input [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: [s_store_sk#7, s_county#8, s_state#21] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3] -(33) CometBroadcastHashJoin -Left output [2]: [ss_store_sk#1, ss_net_profit#2] -Right output [3]: [s_store_sk#7, s_county#8, s_state#21] -Arguments: [ss_store_sk#1], [s_store_sk#7], Inner, BuildRight +(36) BroadcastHashJoin [codegen id : 4] +Left keys [1]: [ss_store_sk#1] +Right keys [1]: [s_store_sk#7] +Join type: Inner +Join condition: None -(34) CometProject +(37) Project [codegen id : 4] +Output [3]: [ss_net_profit#2, s_county#8, s_state#21] Input [5]: [ss_store_sk#1, ss_net_profit#2, s_store_sk#7, s_county#8, s_state#21] -Arguments: [ss_net_profit#2, s_county#8, s_state#21], [ss_net_profit#2, s_county#8, s_state#21] -(35) CometHashAggregate +(38) HashAggregate [codegen id : 4] Input [3]: [ss_net_profit#2, s_county#8, s_state#21] Keys [2]: [s_state#21, s_county#8] Functions [1]: [partial_sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum#22] +Results [3]: [s_state#21, s_county#8, sum#23] -(36) CometExchange -Input [3]: [s_state#21, s_county#8, sum#22] -Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(39) CometColumnarExchange +Input [3]: [s_state#21, s_county#8, sum#23] +Arguments: hashpartitioning(s_state#21, s_county#8, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] -(37) CometHashAggregate -Input [3]: [s_state#21, s_county#8, sum#22] +(40) CometColumnarToRow [codegen id : 5] +Input [3]: [s_state#21, s_county#8, sum#23] + +(41) HashAggregate [codegen id : 5] +Input [3]: [s_state#21, s_county#8, sum#23] Keys [2]: [s_state#21, s_county#8] Functions [1]: [sum(UnscaledValue(ss_net_profit#2))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#2))#24] +Results [6]: [cast(MakeDecimal(sum(UnscaledValue(ss_net_profit#2))#24,17,2) as decimal(27,2)) AS total_sum#25, s_state#21 AS s_state#26, s_county#8 AS s_county#27, 0 AS g_state#28, 0 AS g_county#29, 0 AS lochierarchy#30] + +(42) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_state#21, s_county#31, sum#32] -(38) ReusedExchange [Reuses operator id: 36] -Output [3]: [s_state#21, s_county#23, sum#24] +(43) CometColumnarToRow [codegen id : 10] +Input [3]: [s_state#21, s_county#31, sum#32] -(39) CometHashAggregate -Input [3]: [s_state#21, s_county#23, sum#24] -Keys [2]: [s_state#21, s_county#23] -Functions [1]: [sum(UnscaledValue(ss_net_profit#25))] +(44) HashAggregate [codegen id : 10] +Input [3]: [s_state#21, s_county#31, sum#32] +Keys [2]: [s_state#21, s_county#31] +Functions [1]: [sum(UnscaledValue(ss_net_profit#33))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#33))#24] +Results [2]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#33))#24,17,2) AS total_sum#34, s_state#21] -(40) CometHashAggregate -Input [2]: [total_sum#26, s_state#21] +(45) HashAggregate [codegen id : 10] +Input [2]: [total_sum#34, s_state#21] Keys [1]: [s_state#21] -Functions [1]: [partial_sum(total_sum#26)] +Functions [1]: [partial_sum(total_sum#34)] +Aggregate Attributes [2]: [sum#35, isEmpty#36] +Results [3]: [s_state#21, sum#37, isEmpty#38] -(41) CometExchange -Input [3]: [s_state#21, sum#27, isEmpty#28] -Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +(46) CometColumnarExchange +Input [3]: [s_state#21, sum#37, isEmpty#38] +Arguments: hashpartitioning(s_state#21, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] -(42) CometHashAggregate -Input [3]: [s_state#21, sum#27, isEmpty#28] +(47) CometColumnarToRow [codegen id : 11] +Input [3]: [s_state#21, sum#37, isEmpty#38] + +(48) HashAggregate [codegen id : 11] +Input [3]: [s_state#21, sum#37, isEmpty#38] Keys [1]: [s_state#21] -Functions [1]: [sum(total_sum#26)] +Functions [1]: [sum(total_sum#34)] +Aggregate Attributes [1]: [sum(total_sum#34)#39] +Results [6]: [sum(total_sum#34)#39 AS total_sum#40, s_state#21, null AS s_county#41, 0 AS g_state#42, 1 AS g_county#43, 1 AS lochierarchy#44] + +(49) ReusedExchange [Reuses operator id: 39] +Output [3]: [s_state#21, s_county#45, sum#46] -(43) ReusedExchange [Reuses operator id: 36] -Output [3]: [s_state#21, s_county#29, sum#30] +(50) CometColumnarToRow [codegen id : 16] +Input [3]: [s_state#21, s_county#45, sum#46] -(44) CometHashAggregate -Input [3]: [s_state#21, s_county#29, sum#30] -Keys [2]: [s_state#21, s_county#29] -Functions [1]: [sum(UnscaledValue(ss_net_profit#31))] +(51) HashAggregate [codegen id : 16] +Input [3]: [s_state#21, s_county#45, sum#46] +Keys [2]: [s_state#21, s_county#45] +Functions [1]: [sum(UnscaledValue(ss_net_profit#47))] +Aggregate Attributes [1]: [sum(UnscaledValue(ss_net_profit#47))#24] +Results [1]: [MakeDecimal(sum(UnscaledValue(ss_net_profit#47))#24,17,2) AS total_sum#34] -(45) CometHashAggregate -Input [1]: [total_sum#26] +(52) HashAggregate [codegen id : 16] +Input [1]: [total_sum#34] Keys: [] -Functions [1]: [partial_sum(total_sum#26)] +Functions [1]: [partial_sum(total_sum#34)] +Aggregate Attributes [2]: [sum#48, isEmpty#49] +Results [2]: [sum#50, isEmpty#51] + +(53) CometColumnarExchange +Input [2]: [sum#50, isEmpty#51] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] -(46) CometExchange -Input [2]: [sum#32, isEmpty#33] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(54) CometColumnarToRow [codegen id : 17] +Input [2]: [sum#50, isEmpty#51] -(47) CometHashAggregate -Input [2]: [sum#32, isEmpty#33] +(55) HashAggregate [codegen id : 17] +Input [2]: [sum#50, isEmpty#51] Keys: [] -Functions [1]: [sum(total_sum#26)] +Functions [1]: [sum(total_sum#34)] +Aggregate Attributes [1]: [sum(total_sum#34)#52] +Results [6]: [sum(total_sum#34)#52 AS total_sum#53, null AS s_state#54, null AS s_county#55, 1 AS g_state#56, 1 AS g_county#57, 2 AS lochierarchy#58] -(48) CometUnion -Child 0 Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] -Child 1 Input [6]: [total_sum#40, s_state#21, s_county#41, g_state#42, g_county#43, lochierarchy#44] -Child 2 Input [6]: [total_sum#45, s_state#46, s_county#47, g_state#48, g_county#49, lochierarchy#50] +(56) Union -(49) CometHashAggregate -Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] -Keys [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +(57) HashAggregate [codegen id : 18] +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Functions: [] +Aggregate Attributes: [] +Results [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] -(50) CometExchange -Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] -Arguments: hashpartitioning(total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(58) CometColumnarExchange +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Arguments: hashpartitioning(total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] -(51) CometHashAggregate -Input [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] -Keys [6]: [total_sum#34, s_state#35, s_county#36, g_state#37, g_county#38, lochierarchy#39] +(59) CometHashAggregate +Input [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] +Keys [6]: [total_sum#25, s_state#26, s_county#27, g_state#28, g_county#29, lochierarchy#30] Functions: [] -(52) CometExchange -Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] -Arguments: hashpartitioning(lochierarchy#39, _w0#51, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(60) CometExchange +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] +Arguments: hashpartitioning(lochierarchy#30, _w0#59, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(53) CometSort -Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] -Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51], [lochierarchy#39 ASC NULLS FIRST, _w0#51 ASC NULLS FIRST, total_sum#34 DESC NULLS LAST] +(61) CometSort +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] +Arguments: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59], [lochierarchy#30 ASC NULLS FIRST, _w0#59 ASC NULLS FIRST, total_sum#25 DESC NULLS LAST] -(54) CometWindowExec -Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51] -Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51, rank_within_parent#52], [rank(total_sum#34) windowspecdefinition(lochierarchy#39, _w0#51, total_sum#34 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#52], [lochierarchy#39, _w0#51], [total_sum#34 DESC NULLS LAST] +(62) CometColumnarToRow [codegen id : 19] +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] -(55) CometProject -Input [6]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, _w0#51, rank_within_parent#52] -Arguments: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52], [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] +(63) Window +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59] +Arguments: [rank(total_sum#25) windowspecdefinition(lochierarchy#30, _w0#59, total_sum#25 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#60], [lochierarchy#30, _w0#59], [total_sum#25 DESC NULLS LAST] -(56) CometTakeOrderedAndProject -Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#39 DESC NULLS LAST,CASE WHEN (lochierarchy#39 = 0) THEN s_state#35 END ASC NULLS FIRST,rank_within_parent#52 ASC NULLS FIRST], output=[total_sum#34,s_state#35,s_county#36,lochierarchy#39,rank_within_parent#52]), [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52], 100, 0, [lochierarchy#39 DESC NULLS LAST, CASE WHEN (lochierarchy#39 = 0) THEN s_state#35 END ASC NULLS FIRST, rank_within_parent#52 ASC NULLS FIRST], [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] +(64) Project [codegen id : 20] +Output [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] +Input [6]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, _w0#59, rank_within_parent#60] -(57) CometColumnarToRow [codegen id : 1] -Input [5]: [total_sum#34, s_state#35, s_county#36, lochierarchy#39, rank_within_parent#52] +(65) TakeOrderedAndProject +Input [5]: [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] +Arguments: 100, [lochierarchy#30 DESC NULLS LAST, CASE WHEN (lochierarchy#30 = 0) THEN s_state#26 END ASC NULLS FIRST, rank_within_parent#60 ASC NULLS FIRST], [total_sum#25, s_state#26, s_county#27, lochierarchy#30, rank_within_parent#60] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (62) -+- * CometColumnarToRow (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (58) +BroadcastExchange (70) ++- * CometColumnarToRow (69) + +- CometProject (68) + +- CometFilter (67) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (66) -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(66) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#5, d_month_seq#6] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct -(59) CometFilter +(67) CometFilter Input [2]: [d_date_sk#5, d_month_seq#6] Condition : (((isnotnull(d_month_seq#6) AND (d_month_seq#6 >= 1212)) AND (d_month_seq#6 <= 1223)) AND isnotnull(d_date_sk#5)) -(60) CometProject +(68) CometProject Input [2]: [d_date_sk#5, d_month_seq#6] Arguments: [d_date_sk#5], [d_date_sk#5] -(61) CometColumnarToRow [codegen id : 1] +(69) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#5] -(62) BroadcastExchange +(70) BroadcastExchange Input [1]: [d_date_sk#5] -Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7] +Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=9] -Subquery:2 Hosting operator id = 11 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 +Subquery:2 Hosting operator id = 13 Hosting Expression = ss_sold_date_sk#12 IN dynamicpruning#4 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt index 5b14945482..a187c52d0d 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/extended.txt @@ -1,159 +1,173 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometUnion - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometHashAggregate - : +- CometExchange + +- CometColumnarExchange + +- HashAggregate + +- Union + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - :- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- SubqueryBroadcast - : : : +- BroadcastExchange - : : : +- CometColumnarToRow - : : : +- CometProject - : : : +- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometWindowExec - : +- CometSort - : +- CometHashAggregate - : +- CometExchange - : +- CometHashAggregate - : +- CometProject - : +- CometBroadcastHashJoin - : :- CometProject - : : +- CometBroadcastHashJoin - : : :- CometFilter - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : : +- ReusedSubquery - : : +- CometBroadcastExchange - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - : +- CometBroadcastExchange + : +- CometExchange + : +- CometHashAggregate + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + :- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- HashAggregate + : +- CometColumnarToRow + : +- CometColumnarExchange + : +- HashAggregate + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- SubqueryBroadcast + : : : +- BroadcastExchange + : : : +- CometColumnarToRow + : : : +- CometProject + : : : +- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- BroadcastExchange + : +- Project + : +- BroadcastHashJoin + : :- CometColumnarToRow + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- BroadcastExchange + : +- Project + : +- Filter + : +- Window [COMET: Window expressions are not supported] + : +- CometColumnarToRow + : +- CometSort + : +- CometHashAggregate + : +- CometExchange + : +- CometHashAggregate : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - +- CometBroadcastExchange - +- CometProject - +- CometBroadcastHashJoin - :- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometWindowExec - +- CometSort - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- ReusedSubquery - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store - +- CometBroadcastExchange + : +- CometBroadcastHashJoin + : :- CometProject + : : +- CometBroadcastHashJoin + : : :- CometFilter + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : : +- ReusedSubquery + : : +- CometBroadcastExchange + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- HashAggregate + +- CometColumnarToRow + +- CometColumnarExchange + +- HashAggregate + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- BroadcastExchange + +- Project + +- BroadcastHashJoin + :- CometColumnarToRow + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- BroadcastExchange + +- Project + +- Filter + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- ReusedSubquery + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 144 out of 153 eligible operators (94%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 102 out of 153 eligible operators (66%). Final plan contains 18 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt index 91be7bc024..8de8b7e637 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q70a/simplified.txt @@ -1,68 +1,100 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometProject [total_sum,s_state,s_county,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,s_state,s_county,lochierarchy,_w0,rank_within_parent] - CometSort [total_sum,s_state,s_county,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] - CometExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 - CometHashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - CometUnion [total_sum,s_state,s_county,g_state,g_county,lochierarchy] - CometHashAggregate [sum] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(UnscaledValue(ss_net_profit)),s_state,s_county] - CometExchange [s_state,s_county] #3 - CometHashAggregate [ss_net_profit] [s_state,s_county,sum] - CometProject [ss_net_profit,s_county,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,s_store_sk,s_county,s_state] - CometProject [ss_store_sk,ss_net_profit] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) +TakeOrderedAndProject [lochierarchy,s_state,rank_within_parent,total_sum,s_county] + WholeStageCodegen (20) + Project [total_sum,s_state,s_county,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (19) + CometColumnarToRow + InputAdapter + CometSort [total_sum,s_state,s_county,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_state] [total_sum,s_state,s_county,lochierarchy,_w0,g_county] + CometColumnarExchange [total_sum,s_state,s_county,g_state,g_county,lochierarchy] #2 + WholeStageCodegen (18) + HashAggregate [total_sum,s_state,s_county,g_state,g_county,lochierarchy] + InputAdapter + Union + WholeStageCodegen (5) + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state,s_county] #3 + WholeStageCodegen (4) + HashAggregate [s_state,s_county,ss_net_profit] [sum,sum] + Project [ss_net_profit,s_county,s_state] + BroadcastHashJoin [ss_store_sk,s_store_sk] + CometColumnarToRow + InputAdapter + CometProject [ss_store_sk,ss_net_profit] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,d_date_sk] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + InputAdapter + BroadcastExchange #6 + WholeStageCodegen (3) + Project [s_store_sk,s_county,s_state] + BroadcastHashJoin [s_state,s_state] + CometColumnarToRow + InputAdapter + CometFilter [s_store_sk,s_county,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] + InputAdapter + BroadcastExchange #7 + WholeStageCodegen (2) + Project [s_state] + Filter [ranking] + InputAdapter + Window [_w0,s_state] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [s_state,_w0] + CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] + CometExchange [s_state] #8 + CometHashAggregate [ss_net_profit] [s_state,sum] + CometProject [ss_net_profit,s_state] + CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] + CometProject [ss_net_profit,ss_sold_date_sk,s_state] + CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] + CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [s_store_sk,s_state] #9 + CometProject [s_state] [s_store_sk,s_state] + CometFilter [s_store_sk,s_state] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] + ReusedExchange [d_date_sk] #5 + WholeStageCodegen (11) + HashAggregate [s_state,sum,isEmpty] [sum(total_sum),total_sum,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange [s_state] #10 + WholeStageCodegen (10) + HashAggregate [s_state,total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] + CometColumnarToRow + InputAdapter + ReusedExchange [s_state,s_county,sum] #3 + WholeStageCodegen (17) + HashAggregate [sum,isEmpty] [sum(total_sum),total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum,isEmpty] + CometColumnarToRow + InputAdapter + CometColumnarExchange #11 + WholeStageCodegen (16) + HashAggregate [total_sum] [sum,isEmpty,sum,isEmpty] + HashAggregate [s_state,s_county,sum] [sum(UnscaledValue(ss_net_profit)),total_sum,sum] CometColumnarToRow InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [s_store_sk,s_county,s_state] #6 - CometProject [s_state] [s_store_sk,s_county,s_state] - CometBroadcastHashJoin [s_store_sk,s_county,s_state,s_state] - CometFilter [s_store_sk,s_county,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_county,s_state] - CometBroadcastExchange [s_state] #7 - CometProject [s_state] - CometFilter [s_state,_w0,ranking] - CometWindowExec [s_state,_w0,ranking] - CometSort [s_state,_w0] - CometHashAggregate [sum] [s_state,_w0,sum(UnscaledValue(ss_net_profit))] - CometExchange [s_state] #8 - CometHashAggregate [ss_net_profit] [s_state,sum] - CometProject [ss_net_profit,s_state] - CometBroadcastHashJoin [ss_net_profit,ss_sold_date_sk,s_state,d_date_sk] - CometProject [ss_net_profit,ss_sold_date_sk,s_state] - CometBroadcastHashJoin [ss_store_sk,ss_net_profit,ss_sold_date_sk,s_store_sk,s_state] - CometFilter [ss_store_sk,ss_net_profit,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_store_sk,ss_net_profit,ss_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [s_store_sk,s_state] #9 - CometProject [s_state] [s_store_sk,s_state] - CometFilter [s_store_sk,s_state] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store [s_store_sk,s_state] - ReusedExchange [d_date_sk] #5 - CometHashAggregate [sum,isEmpty] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(total_sum)] - CometExchange [s_state] #10 - CometHashAggregate [total_sum] [s_state,sum,isEmpty] - CometHashAggregate [s_county,sum] [total_sum,s_state,sum(UnscaledValue(ss_net_profit))] - ReusedExchange [s_state,s_county,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,s_state,s_county,g_state,g_county,lochierarchy,sum(total_sum)] - CometExchange #11 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [s_state,s_county,sum] [total_sum,sum(UnscaledValue(ss_net_profit))] - ReusedExchange [s_state,s_county,sum] #3 + ReusedExchange [s_state,s_county,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt index 5269452a9e..c05e23f926 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (37) -+- CometTakeOrderedAndProject (36) - +- CometProject (35) - +- CometWindowExec (34) +TakeOrderedAndProject (37) ++- * Project (36) + +- Window (35) + +- * CometColumnarToRow (34) +- CometSort (33) +- CometExchange (32) +- CometHashAggregate (31) @@ -191,20 +191,20 @@ Arguments: hashpartitioning(lochierarchy#27, _w0#39, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39], [lochierarchy#27 ASC NULLS FIRST, _w0#39 ASC NULLS FIRST, total_sum#22 DESC NULLS LAST] -(34) CometWindowExec +(34) CometColumnarToRow [codegen id : 1] Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] -Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40], [rank(total_sum#22) windowspecdefinition(lochierarchy#27, _w0#39, total_sum#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#40], [lochierarchy#27, _w0#39], [total_sum#22 DESC NULLS LAST] -(35) CometProject -Input [6]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40] -Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +(35) Window +Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] +Arguments: [rank(total_sum#22) windowspecdefinition(lochierarchy#27, _w0#39, total_sum#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#40], [lochierarchy#27, _w0#39], [total_sum#22 DESC NULLS LAST] -(36) CometTakeOrderedAndProject -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#27 DESC NULLS LAST,CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST,rank_within_parent#40 ASC NULLS FIRST], output=[total_sum#22,i_category#23,i_class#24,lochierarchy#27,rank_within_parent#40]), [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40], 100, 0, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST, rank_within_parent#40 ASC NULLS FIRST], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +(36) Project [codegen id : 2] +Output [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +Input [6]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40] -(37) CometColumnarToRow [codegen id : 1] +(37) TakeOrderedAndProject Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST, rank_within_parent#40 ASC NULLS FIRST], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt index 91a7f81241..6dd65034ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -84,4 +84,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt index ad8ab5e34a..11e3c03d80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a.native_iceberg_compat/simplified.txt @@ -1,47 +1,49 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,rank_within_parent] - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (2) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange [i_category] #7 + CometHashAggregate [total_sum] [i_category,sum,isEmpty] + CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange #8 + CometHashAggregate [total_sum] [sum,isEmpty] + CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt index 5269452a9e..c05e23f926 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/explain.txt @@ -1,8 +1,8 @@ == Physical Plan == -* CometColumnarToRow (37) -+- CometTakeOrderedAndProject (36) - +- CometProject (35) - +- CometWindowExec (34) +TakeOrderedAndProject (37) ++- * Project (36) + +- Window (35) + +- * CometColumnarToRow (34) +- CometSort (33) +- CometExchange (32) +- CometHashAggregate (31) @@ -191,20 +191,20 @@ Arguments: hashpartitioning(lochierarchy#27, _w0#39, 5), ENSURE_REQUIREMENTS, Co Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39], [lochierarchy#27 ASC NULLS FIRST, _w0#39 ASC NULLS FIRST, total_sum#22 DESC NULLS LAST] -(34) CometWindowExec +(34) CometColumnarToRow [codegen id : 1] Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] -Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40], [rank(total_sum#22) windowspecdefinition(lochierarchy#27, _w0#39, total_sum#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#40], [lochierarchy#27, _w0#39], [total_sum#22 DESC NULLS LAST] -(35) CometProject -Input [6]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40] -Arguments: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +(35) Window +Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39] +Arguments: [rank(total_sum#22) windowspecdefinition(lochierarchy#27, _w0#39, total_sum#22 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rank_within_parent#40], [lochierarchy#27, _w0#39], [total_sum#22 DESC NULLS LAST] -(36) CometTakeOrderedAndProject -Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[lochierarchy#27 DESC NULLS LAST,CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST,rank_within_parent#40 ASC NULLS FIRST], output=[total_sum#22,i_category#23,i_class#24,lochierarchy#27,rank_within_parent#40]), [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40], 100, 0, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST, rank_within_parent#40 ASC NULLS FIRST], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +(36) Project [codegen id : 2] +Output [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +Input [6]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, _w0#39, rank_within_parent#40] -(37) CometColumnarToRow [codegen id : 1] +(37) TakeOrderedAndProject Input [5]: [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] +Arguments: 100, [lochierarchy#27 DESC NULLS LAST, CASE WHEN (lochierarchy#27 = 0) THEN i_category#23 END ASC NULLS FIRST, rank_within_parent#40 ASC NULLS FIRST], [total_sum#22, i_category#23, i_class#24, lochierarchy#27, rank_within_parent#40] ===== Subqueries ===== diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt index 91a7f81241..6dd65034ce 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/extended.txt @@ -1,7 +1,7 @@ -CometColumnarToRow -+- CometTakeOrderedAndProject - +- CometProject - +- CometWindowExec + TakeOrderedAndProject [COMET: ] ++- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow +- CometSort +- CometExchange +- CometHashAggregate @@ -84,4 +84,4 @@ CometColumnarToRow +- CometFilter +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item -Comet accelerated 75 out of 81 eligible operators (92%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 72 out of 81 eligible operators (88%). Final plan contains 4 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt index ad8ab5e34a..11e3c03d80 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q86a/simplified.txt @@ -1,47 +1,49 @@ -WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometTakeOrderedAndProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometProject [total_sum,i_category,i_class,lochierarchy,rank_within_parent] - CometWindowExec [total_sum,i_category,i_class,lochierarchy,_w0,rank_within_parent] - CometSort [total_sum,i_category,i_class,lochierarchy,_w0] - CometExchange [lochierarchy,_w0] #1 - CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] - CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 - CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] - CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] - CometExchange [i_category,i_class] #3 - CometHashAggregate [ws_net_paid] [i_category,i_class,sum] - CometProject [ws_net_paid,i_class,i_category] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] - CometProject [ws_item_sk,ws_net_paid] - CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] - CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [d_date_sk] #5 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_month_seq] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] - CometBroadcastExchange [i_item_sk,i_class,i_category] #6 - CometProject [i_class,i_category] [i_item_sk,i_class,i_category] - CometFilter [i_item_sk,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange [i_category] #7 - CometHashAggregate [total_sum] [i_category,sum,isEmpty] - CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 - CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] - CometExchange #8 - CometHashAggregate [total_sum] [sum,isEmpty] - CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] - ReusedExchange [i_category,i_class,sum] #3 +TakeOrderedAndProject [lochierarchy,i_category,rank_within_parent,total_sum,i_class] + WholeStageCodegen (2) + Project [total_sum,i_category,i_class,lochierarchy,rank_within_parent] + InputAdapter + Window [total_sum,lochierarchy,_w0] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [total_sum,i_category,i_class,lochierarchy,_w0] + CometExchange [lochierarchy,_w0] #1 + CometHashAggregate [g_category] [total_sum,i_category,i_class,lochierarchy,_w0,g_class] + CometExchange [total_sum,i_category,i_class,g_category,g_class,lochierarchy] #2 + CometHashAggregate [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometUnion [total_sum,i_category,i_class,g_category,g_class,lochierarchy] + CometHashAggregate [sum] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(UnscaledValue(ws_net_paid)),i_category,i_class] + CometExchange [i_category,i_class] #3 + CometHashAggregate [ws_net_paid] [i_category,i_class,sum] + CometProject [ws_net_paid,i_class,i_category] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,i_item_sk,i_class,i_category] + CometProject [ws_item_sk,ws_net_paid] + CometBroadcastHashJoin [ws_item_sk,ws_net_paid,ws_sold_date_sk,d_date_sk] + CometFilter [ws_item_sk,ws_net_paid,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_net_paid,ws_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [d_date_sk] #5 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_month_seq] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_month_seq] + CometBroadcastExchange [i_item_sk,i_class,i_category] #6 + CometProject [i_class,i_category] [i_item_sk,i_class,i_category] + CometFilter [i_item_sk,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_class,i_category] + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange [i_category] #7 + CometHashAggregate [total_sum] [i_category,sum,isEmpty] + CometHashAggregate [i_class,sum] [total_sum,i_category,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 + CometHashAggregate [sum,isEmpty] [total_sum,i_category,i_class,g_category,g_class,lochierarchy,sum(total_sum)] + CometExchange #8 + CometHashAggregate [total_sum] [sum,isEmpty] + CometHashAggregate [i_category,i_class,sum] [total_sum,sum(UnscaledValue(ws_net_paid))] + ReusedExchange [i_category,i_class,sum] #3 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt index 93c9d0d56c..96a0ec392e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/explain.txt @@ -1,28 +1,29 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometSort (23) - +- CometExchange (22) - +- CometProject (21) - +- CometWindowExec (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (25) ++- CometSort (24) + +- CometColumnarExchange (23) + +- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -115,54 +116,57 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject +(21) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] + +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -(22) CometExchange +(23) CometColumnarExchange Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometSort +(24) CometSort Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 3] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(26) CometFilter +(27) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(27) CometProject +(28) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(28) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(29) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt index 653c70e3d8..54a4d3c1b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/extended.txt @@ -1,32 +1,33 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt index 603b05991a..46191f59cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98.native_iceberg_compat/simplified.txt @@ -1,34 +1,39 @@ -WholeStageCodegen (1) +WholeStageCodegen (3) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt index 93c9d0d56c..96a0ec392e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/explain.txt @@ -1,28 +1,29 @@ == Physical Plan == -* CometColumnarToRow (24) -+- CometSort (23) - +- CometExchange (22) - +- CometProject (21) - +- CometWindowExec (20) - +- CometSort (19) - +- CometExchange (18) - +- CometHashAggregate (17) - +- CometExchange (16) - +- CometHashAggregate (15) - +- CometProject (14) - +- CometBroadcastHashJoin (13) - :- CometProject (8) - : +- CometBroadcastHashJoin (7) - : :- CometFilter (2) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : +- CometBroadcastExchange (6) - : +- CometProject (5) - : +- CometFilter (4) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) - +- CometBroadcastExchange (12) - +- CometProject (11) - +- CometFilter (10) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) +* CometColumnarToRow (25) ++- CometSort (24) + +- CometColumnarExchange (23) + +- * Project (22) + +- Window (21) + +- * CometColumnarToRow (20) + +- CometSort (19) + +- CometExchange (18) + +- CometHashAggregate (17) + +- CometExchange (16) + +- CometHashAggregate (15) + +- CometProject (14) + +- CometBroadcastHashJoin (13) + :- CometProject (8) + : +- CometBroadcastHashJoin (7) + : :- CometFilter (2) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : +- CometBroadcastExchange (6) + : +- CometProject (5) + : +- CometFilter (4) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (3) + +- CometBroadcastExchange (12) + +- CometProject (11) + +- CometFilter (10) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (9) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -115,54 +116,57 @@ Arguments: hashpartitioning(i_class#12, 5), ENSURE_REQUIREMENTS, CometNativeShuf Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18], [i_class#12 ASC NULLS FIRST] -(20) CometWindowExec +(20) CometColumnarToRow [codegen id : 1] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19], [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] -(21) CometProject +(21) Window +Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18] +Arguments: [sum(_w0#18) windowspecdefinition(i_class#12, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#19], [i_class#12] + +(22) Project [codegen id : 2] +Output [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] Input [8]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, _w0#18, _we0#19] -Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, ((_w0#18 * 100) / _we0#19) AS revenueratio#20] -(22) CometExchange +(23) CometColumnarExchange Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] -Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] +Arguments: rangepartitioning(i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=3] -(23) CometSort +(24) CometSort Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] Arguments: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20], [i_category#13 ASC NULLS FIRST, i_class#12 ASC NULLS FIRST, i_item_id#11 ASC NULLS FIRST, i_item_desc#7 ASC NULLS FIRST, revenueratio#20 ASC NULLS FIRST] -(24) CometColumnarToRow [codegen id : 1] +(25) CometColumnarToRow [codegen id : 3] Input [7]: [i_item_id#11, i_item_desc#7, i_category#13, i_class#12, i_current_price#8, itemrevenue#17, revenueratio#20] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4 -BroadcastExchange (29) -+- * CometColumnarToRow (28) - +- CometProject (27) - +- CometFilter (26) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (25) +BroadcastExchange (30) ++- * CometColumnarToRow (29) + +- CometProject (28) + +- CometFilter (27) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (26) -(25) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#14, d_date#15] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1999-02-22), LessThanOrEqual(d_date,1999-03-24), IsNotNull(d_date_sk)] ReadSchema: struct -(26) CometFilter +(27) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1999-02-22)) AND (d_date#15 <= 1999-03-24)) AND isnotnull(d_date_sk#14)) -(27) CometProject +(28) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(28) CometColumnarToRow [codegen id : 1] +(29) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(29) BroadcastExchange +(30) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt index 653c70e3d8..54a4d3c1b7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/extended.txt @@ -1,32 +1,33 @@ CometColumnarToRow +- CometSort - +- CometExchange - +- CometProject - +- CometWindowExec - +- CometSort - +- CometExchange - +- CometHashAggregate - +- CometExchange - +- CometHashAggregate - +- CometProject - +- CometBroadcastHashJoin - :- CometProject - : +- CometBroadcastHashJoin - : :- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales - : : +- SubqueryBroadcast - : : +- BroadcastExchange - : : +- CometColumnarToRow - : : +- CometProject - : : +- CometFilter - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim - : +- CometBroadcastExchange - : +- CometProject - : +- CometFilter - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item - +- CometBroadcastExchange - +- CometProject - +- CometFilter - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + +- CometColumnarExchange + +- Project + +- Window [COMET: Window expressions are not supported] + +- CometColumnarToRow + +- CometSort + +- CometExchange + +- CometHashAggregate + +- CometExchange + +- CometHashAggregate + +- CometProject + +- CometBroadcastHashJoin + :- CometProject + : +- CometBroadcastHashJoin + : :- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales + : : +- SubqueryBroadcast + : : +- BroadcastExchange + : : +- CometColumnarToRow + : : +- CometProject + : : +- CometFilter + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim + : +- CometBroadcastExchange + : +- CometProject + : +- CometFilter + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item + +- CometBroadcastExchange + +- CometProject + +- CometFilter + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim -Comet accelerated 26 out of 28 eligible operators (92%). Final plan contains 2 transitions between Spark and Comet. \ No newline at end of file +Comet accelerated 24 out of 28 eligible operators (85%). Final plan contains 3 transitions between Spark and Comet. \ No newline at end of file diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt index 603b05991a..46191f59cd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q98/simplified.txt @@ -1,34 +1,39 @@ -WholeStageCodegen (1) +WholeStageCodegen (3) CometColumnarToRow InputAdapter CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 - CometProject [_w0,_we0] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,revenueratio] - CometWindowExec [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] - CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] - CometExchange [i_class] #2 - CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] - CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 - CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] - CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] - CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #4 - WholeStageCodegen (1) - CometColumnarToRow - InputAdapter - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 - CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] - CometBroadcastExchange [d_date_sk] #6 - CometProject [d_date_sk] - CometFilter [d_date_sk,d_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometColumnarExchange [i_category,i_class,i_item_id,i_item_desc,revenueratio] #1 + WholeStageCodegen (2) + Project [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,_we0] + InputAdapter + Window [_w0,i_class] + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometSort [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0] + CometExchange [i_class] #2 + CometHashAggregate [sum] [i_item_id,i_item_desc,i_category,i_class,i_current_price,itemrevenue,_w0,sum(UnscaledValue(ss_ext_sales_price))] + CometExchange [i_item_id,i_item_desc,i_category,i_class,i_current_price] #3 + CometHashAggregate [ss_ext_sales_price] [i_item_id,i_item_desc,i_category,i_class,i_current_price,sum] + CometProject [ss_ext_sales_price,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category,d_date_sk] + CometProject [ss_ext_sales_price,ss_sold_date_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastHashJoin [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk,i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales [ss_item_sk,ss_ext_sales_price,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #4 + WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometBroadcastExchange [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] #5 + CometProject [i_item_id,i_class,i_category] [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometFilter [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_item_id,i_item_desc,i_current_price,i_class,i_category] + CometBroadcastExchange [d_date_sk] #6 + CometProject [d_date_sk] + CometFilter [d_date_sk,d_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] From 928e22321a0d6ada36b185191a2d975d446fb858 Mon Sep 17 00:00:00 2001 From: comphead Date: Fri, 7 Nov 2025 20:43:42 -0800 Subject: [PATCH 5/6] chore: Fallback to Spark for windows --- common/src/main/scala/org/apache/comet/CometConf.scala | 2 +- .../org/apache/comet/exec/CometWindowExecSuite.scala | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index d48d149728..d44a19a072 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -251,7 +251,7 @@ object CometConf extends ShimCometConf { val COMET_EXEC_EXPAND_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("expand", defaultValue = true) val COMET_EXEC_WINDOW_ENABLED: ConfigEntry[Boolean] = - createExecEnabledConfig("window", defaultValue = true) + createExecEnabledConfig("window", defaultValue = false) val COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("takeOrderedAndProject", defaultValue = true) diff --git a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala index a0d3d733ff..60f2b9b500 100644 --- a/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala +++ b/spark/src/test/scala/org/apache/comet/exec/CometWindowExecSuite.scala @@ -258,7 +258,7 @@ class CometWindowExecSuite extends CometTestBase { } } - test("Windows support") { + ignore("Windows support") { Seq("true", "false").foreach(aqeEnabled => withSQLConf( CometConf.COMET_EXEC_SHUFFLE_ENABLED.key -> "true", @@ -284,7 +284,7 @@ class CometWindowExecSuite extends CometTestBase { s"SELECT $function OVER(order by _2 rows between current row and 1 following) FROM t1") queries.foreach { query => - checkSparkAnswerAndOperator(query) + checkSparkAnswerAndFallbackReason(query, "Window expressions are not supported") } } } @@ -303,7 +303,7 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql("SELECT a, b, c, COUNT(*) OVER () as cnt FROM window_test") - checkSparkAnswerAndOperator(df) + checkSparkAnswerAndFallbackReason(df, "Window expressions are not supported") } } @@ -319,7 +319,7 @@ class CometWindowExecSuite extends CometTestBase { spark.read.parquet(dir.toString).createOrReplaceTempView("window_test") val df = sql("SELECT a, b, c, SUM(c) OVER (PARTITION BY a) as sum_c FROM window_test") - checkSparkAnswerAndOperator(df) + checkSparkAnswerAndFallbackReason(df, "Window expressions are not supported") } } @@ -359,7 +359,7 @@ class CometWindowExecSuite extends CometTestBase { MAX(c) OVER (ORDER BY b) as max_c FROM window_test """) - checkSparkAnswerAndOperator(df) + checkSparkAnswerAndFallbackReason(df, "Window expressions are not supported") } } From 926491c142ad24dcd4469eed7be4d48e170fa3b1 Mon Sep 17 00:00:00 2001 From: comphead Date: Fri, 7 Nov 2025 21:34:14 -0800 Subject: [PATCH 6/6] chore: Fallback to Spark for windows --- common/src/main/scala/org/apache/comet/CometConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/src/main/scala/org/apache/comet/CometConf.scala b/common/src/main/scala/org/apache/comet/CometConf.scala index d44a19a072..d48d149728 100644 --- a/common/src/main/scala/org/apache/comet/CometConf.scala +++ b/common/src/main/scala/org/apache/comet/CometConf.scala @@ -251,7 +251,7 @@ object CometConf extends ShimCometConf { val COMET_EXEC_EXPAND_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("expand", defaultValue = true) val COMET_EXEC_WINDOW_ENABLED: ConfigEntry[Boolean] = - createExecEnabledConfig("window", defaultValue = false) + createExecEnabledConfig("window", defaultValue = true) val COMET_EXEC_TAKE_ORDERED_AND_PROJECT_ENABLED: ConfigEntry[Boolean] = createExecEnabledConfig("takeOrderedAndProject", defaultValue = true)