From 433edb27205dc2087f9466322c9507c0cec46f77 Mon Sep 17 00:00:00 2001 From: comphead Date: Sat, 18 Oct 2025 10:35:46 -0700 Subject: [PATCH 1/8] feat: support `concat` --- .../scala/org/apache/comet/serde/QueryPlanSerde.scala | 1 + .../scala/org/apache/comet/CometExpressionSuite.scala | 11 +++++++++++ 2 files changed, 12 insertions(+) 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 6dd4548268..655f341fc2 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -164,6 +164,7 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[BitLength] -> CometScalarFunction("bit_length"), classOf[Chr] -> CometScalarFunction("char"), classOf[ConcatWs] -> CometScalarFunction("concat_ws"), + classOf[Concat] -> CometScalarFunction("concat"), classOf[Contains] -> CometScalarFunction("contains"), classOf[EndsWith] -> CometScalarFunction("ends_with"), classOf[InitCap] -> CometInitCap, diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index c36d4979ff..306eed55ee 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -3219,4 +3219,15 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } } + test("test concat function - strings") { + withTable("t1") { + sql( + "create table t1 using parquet as select uuid() c1, uuid() c2, uuid() c3, uuid() c4, cast(null as string) c5 from range(10)") + checkSparkAnswerAndOperator("select concat(c1, c2) AS x FROM t1") + checkSparkAnswerAndOperator("select concat(c1, c2, c3) AS x FROM t1") + checkSparkAnswerAndOperator("select concat(c1, c2, c3, c5) AS x FROM t1") + //checkSparkAnswerAndOperator("select concat(concat(c1, c2, c3), concat(c1, c3)) AS x FROM t1") + } + } + } From 85a9c8792649e2246e84974bffb2496f78192a3a Mon Sep 17 00:00:00 2001 From: comphead Date: Sat, 25 Oct 2025 19:18:00 -0700 Subject: [PATCH 2/8] feat: support `concat` for strings --- docs/source/user-guide/latest/configs.md | 1 + native/Cargo.lock | 1 + native/core/src/execution/jni_api.rs | 16 ++++++---- native/spark-expr/Cargo.toml | 1 + native/spark-expr/src/comet_scalar_funcs.rs | 11 +++++++ .../apache/comet/CometExpressionSuite.scala | 31 ++++++++++++++++++- 6 files changed, 54 insertions(+), 7 deletions(-) diff --git a/docs/source/user-guide/latest/configs.md b/docs/source/user-guide/latest/configs.md index 77bdaf1cc7..537d0d7748 100644 --- a/docs/source/user-guide/latest/configs.md +++ b/docs/source/user-guide/latest/configs.md @@ -201,6 +201,7 @@ These settings can be used to determine which parts of the plan are accelerated | `spark.comet.expression.CheckOverflow.enabled` | Enable Comet acceleration for `CheckOverflow` | true | | `spark.comet.expression.Chr.enabled` | Enable Comet acceleration for `Chr` | true | | `spark.comet.expression.Coalesce.enabled` | Enable Comet acceleration for `Coalesce` | true | +| `spark.comet.expression.Concat.enabled` | Enable Comet acceleration for `Concat` | true | | `spark.comet.expression.ConcatWs.enabled` | Enable Comet acceleration for `ConcatWs` | true | | `spark.comet.expression.Contains.enabled` | Enable Comet acceleration for `Contains` | true | | `spark.comet.expression.Cos.enabled` | Enable Comet acceleration for `Cos` | true | diff --git a/native/Cargo.lock b/native/Cargo.lock index 4c8a41344e..c502cbade4 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1590,6 +1590,7 @@ dependencies = [ "chrono-tz", "criterion", "datafusion", + "datafusion-spark", "futures", "hex", "num", diff --git a/native/core/src/execution/jni_api.rs b/native/core/src/execution/jni_api.rs index 716d20e98d..8d76d2c0e1 100644 --- a/native/core/src/execution/jni_api.rs +++ b/native/core/src/execution/jni_api.rs @@ -47,6 +47,7 @@ use datafusion_spark::function::hash::sha1::SparkSha1; use datafusion_spark::function::hash::sha2::SparkSha2; use datafusion_spark::function::math::expm1::SparkExpm1; use datafusion_spark::function::string::char::CharFunc; +use datafusion_spark::function::string::concat::SparkConcat; use futures::poll; use futures::stream::StreamExt; use jni::objects::JByteBuffer; @@ -317,8 +318,15 @@ fn prepare_datafusion_session_context( let mut session_ctx = SessionContext::new_with_config_rt(session_config, Arc::new(runtime)); datafusion::functions_nested::register_all(&mut session_ctx)?; + register_datafusion_spark_function(&session_ctx); + // Must be the last one to override existing functions with the same name + datafusion_comet_spark_expr::register_all_comet_functions(&mut session_ctx)?; + + Ok(session_ctx) +} - // register UDFs from datafusion-spark crate +// register UDFs from datafusion-spark crate +fn register_datafusion_spark_function(session_ctx: &SessionContext) { session_ctx.register_udf(ScalarUDF::new_from_impl(SparkExpm1::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha2::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(CharFunc::default())); @@ -326,11 +334,7 @@ fn prepare_datafusion_session_context( session_ctx.register_udf(ScalarUDF::new_from_impl(SparkDateAdd::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkDateSub::default())); session_ctx.register_udf(ScalarUDF::new_from_impl(SparkSha1::default())); - - // Must be the last one to override existing functions with the same name - datafusion_comet_spark_expr::register_all_comet_functions(&mut session_ctx)?; - - Ok(session_ctx) + session_ctx.register_udf(ScalarUDF::new_from_impl(SparkConcat::default())); } /// Prepares arrow arrays for output. diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index 961c7bec08..2fb6f6a3a3 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -30,6 +30,7 @@ edition = { workspace = true } arrow = { workspace = true } chrono = { workspace = true } datafusion = { workspace = true } +datafusion-spark = { workspace = true } chrono-tz = { workspace = true } num = { workspace = true } regex = { workspace = true } diff --git a/native/spark-expr/src/comet_scalar_funcs.rs b/native/spark-expr/src/comet_scalar_funcs.rs index fc0c096b15..a54d7e3165 100644 --- a/native/spark-expr/src/comet_scalar_funcs.rs +++ b/native/spark-expr/src/comet_scalar_funcs.rs @@ -180,6 +180,17 @@ pub fn create_comet_physical_fun_with_eval_mode( let func = Arc::new(spark_modulo); make_comet_scalar_udf!("spark_modulo", func, without data_type, fail_on_error) } + "concat" => { + // Use concat from datafusion-spark crate which has Spark semantics + // (returns null if any argument is null) + // The registry parameter already contains functions from datafusion-spark + // if they were registered in prepare_datafusion_session_context + registry.udf("concat").map_err(|e| { + DataFusionError::Execution(format!( + "Function concat not found in the registry: {e}", + )) + }) + } _ => registry.udf(fun_name).map_err(|e| { DataFusionError::Execution(format!( "Function {fun_name} not found in the registry: {e}", diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 306eed55ee..2c3bb3647a 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -3224,10 +3224,39 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { sql( "create table t1 using parquet as select uuid() c1, uuid() c2, uuid() c3, uuid() c4, cast(null as string) c5 from range(10)") checkSparkAnswerAndOperator("select concat(c1, c2) AS x FROM t1") + checkSparkAnswerAndOperator("select concat(c1, c1) AS x FROM t1") checkSparkAnswerAndOperator("select concat(c1, c2, c3) AS x FROM t1") checkSparkAnswerAndOperator("select concat(c1, c2, c3, c5) AS x FROM t1") - //checkSparkAnswerAndOperator("select concat(concat(c1, c2, c3), concat(c1, c3)) AS x FROM t1") + checkSparkAnswerAndOperator( + "select concat(concat(c1, c2, c3), concat(c1, c3)) AS x FROM t1") + } + } + + // https://github.com/apache/datafusion-comet/issues/2647 + ignore("test concat function - arrays") { + withTable("t1") { + sql( + "create table t1 using parquet as select array(id, id+1) c1, array(id+2, id+3) c2, array() c3, array(null) c4, cast(null as array) c5 from range(10)") + checkSparkAnswerAndOperator("select concat(c1, c2) AS x FROM t1") + checkSparkAnswerAndOperator("select concat(c1, c1) AS x FROM t1") + checkSparkAnswerAndOperator("select concat(c1, c2, c3) AS x FROM t1") + checkSparkAnswerAndOperator("select concat(c1, c2, c3, c5) AS x FROM t1") + checkSparkAnswerAndOperator( + "select concat(concat(c1, c2, c3), concat(c1, c3)) AS x FROM t1") } } + // https://github.com/apache/datafusion-comet/issues/2647 + ignore("test concat function - binary") { + withTable("t1") { + sql( + "create table t1 using parquet as select cast(uuid() as binary) c1, cast(uuid() as binary) c2, cast(uuid() as binary) c3, cast(uuid() as binary) c4, cast(null as binary) c5 from range(10)") + checkSparkAnswerAndOperator("select concat(c1, c2) AS x FROM t1") + checkSparkAnswerAndOperator("select concat(c1, c1) AS x FROM t1") + checkSparkAnswerAndOperator("select concat(c1, c2, c3) AS x FROM t1") + checkSparkAnswerAndOperator("select concat(c1, c2, c3, c5) AS x FROM t1") + checkSparkAnswerAndOperator( + "select concat(concat(c1, c2, c3), concat(c1, c3)) AS x FROM t1") + } + } } From 1c4215aafa71c7d4916a52affed9bedf1be31859 Mon Sep 17 00:00:00 2001 From: comphead Date: Sat, 25 Oct 2025 19:25:39 -0700 Subject: [PATCH 3/8] feat: support `concat` for strings --- .../src/main/scala/org/apache/comet/fuzz/Meta.scala | 1 + native/Cargo.lock | 1 - native/spark-expr/Cargo.toml | 1 - native/spark-expr/src/comet_scalar_funcs.rs | 11 ----------- 4 files changed, 1 insertion(+), 13 deletions(-) diff --git a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala index 74d13f85ee..8c2c14e316 100644 --- a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala +++ b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala @@ -137,6 +137,7 @@ object Meta { SparkBinaryType, SparkArrayType( SparkTypeOneOf(Seq(SparkStringType, SparkNumericType, SparkBinaryType))))))), + createFunctionWithInputTypes("concat", Seq(SparkStringType, SparkStringType)), // TODO: variadic createFunctionWithInputTypes("concat_ws", Seq(SparkStringType, SparkStringType)), createFunctionWithInputTypes("contains", Seq(SparkStringType, SparkStringType)), createFunctionWithInputTypes("ends_with", Seq(SparkStringType, SparkStringType)), diff --git a/native/Cargo.lock b/native/Cargo.lock index c502cbade4..4c8a41344e 100644 --- a/native/Cargo.lock +++ b/native/Cargo.lock @@ -1590,7 +1590,6 @@ dependencies = [ "chrono-tz", "criterion", "datafusion", - "datafusion-spark", "futures", "hex", "num", diff --git a/native/spark-expr/Cargo.toml b/native/spark-expr/Cargo.toml index 2fb6f6a3a3..961c7bec08 100644 --- a/native/spark-expr/Cargo.toml +++ b/native/spark-expr/Cargo.toml @@ -30,7 +30,6 @@ edition = { workspace = true } arrow = { workspace = true } chrono = { workspace = true } datafusion = { workspace = true } -datafusion-spark = { workspace = true } chrono-tz = { workspace = true } num = { workspace = true } regex = { workspace = true } diff --git a/native/spark-expr/src/comet_scalar_funcs.rs b/native/spark-expr/src/comet_scalar_funcs.rs index a54d7e3165..fc0c096b15 100644 --- a/native/spark-expr/src/comet_scalar_funcs.rs +++ b/native/spark-expr/src/comet_scalar_funcs.rs @@ -180,17 +180,6 @@ pub fn create_comet_physical_fun_with_eval_mode( let func = Arc::new(spark_modulo); make_comet_scalar_udf!("spark_modulo", func, without data_type, fail_on_error) } - "concat" => { - // Use concat from datafusion-spark crate which has Spark semantics - // (returns null if any argument is null) - // The registry parameter already contains functions from datafusion-spark - // if they were registered in prepare_datafusion_session_context - registry.udf("concat").map_err(|e| { - DataFusionError::Execution(format!( - "Function concat not found in the registry: {e}", - )) - }) - } _ => registry.udf(fun_name).map_err(|e| { DataFusionError::Execution(format!( "Function {fun_name} not found in the registry: {e}", From e0ca43ed0e1db41456a08c6a637a0c715df71f8d Mon Sep 17 00:00:00 2001 From: comphead Date: Sat, 25 Oct 2025 20:48:01 -0700 Subject: [PATCH 4/8] feat: support `concat` for strings --- fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala index 8c2c14e316..9948c428b2 100644 --- a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala +++ b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala @@ -137,7 +137,10 @@ object Meta { SparkBinaryType, SparkArrayType( SparkTypeOneOf(Seq(SparkStringType, SparkNumericType, SparkBinaryType))))))), - createFunctionWithInputTypes("concat", Seq(SparkStringType, SparkStringType)), // TODO: variadic + createFunctionWithInputTypes( + "concat", + Seq(SparkStringType, SparkStringType) + ), // TODO: variadic createFunctionWithInputTypes("concat_ws", Seq(SparkStringType, SparkStringType)), createFunctionWithInputTypes("contains", Seq(SparkStringType, SparkStringType)), createFunctionWithInputTypes("ends_with", Seq(SparkStringType, SparkStringType)), From 08d95c691672d314258874086632f413cae44971 Mon Sep 17 00:00:00 2001 From: comphead Date: Mon, 27 Oct 2025 09:27:01 -0700 Subject: [PATCH 5/8] feat: support `concat` for strings --- .../q5.native_iceberg_compat/explain.txt | 513 +++++++------- .../q5.native_iceberg_compat/simplified.txt | 183 +++-- .../q5/explain.txt | 513 +++++++------- .../q5/simplified.txt | 183 +++-- .../q80.native_iceberg_compat/explain.txt | 669 +++++++++--------- .../q80.native_iceberg_compat/simplified.txt | 241 +++---- .../q80/explain.txt | 669 +++++++++--------- .../q80/simplified.txt | 241 +++---- .../q84.native_iceberg_compat/explain.txt | 20 +- .../q84.native_iceberg_compat/simplified.txt | 10 +- .../q84/explain.txt | 20 +- .../q84/simplified.txt | 10 +- 12 files changed, 1576 insertions(+), 1696 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/explain.txt index f2dce3f729..b4a1dedd61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/explain.txt @@ -1,78 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Expand (69) - +- Union (68) - :- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- * HashAggregate (43) - : +- * CometColumnarToRow (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (33) - : : +- CometBroadcastHashJoin (32) - : : :- CometUnion (30) - : : : :- CometProject (26) - : : : : +- CometFilter (25) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : +- ReusedExchange (31) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - +- * HashAggregate (67) - +- * CometColumnarToRow (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometProject (63) - +- CometBroadcastHashJoin (62) - :- CometProject (57) - : +- CometBroadcastHashJoin (56) - : :- CometUnion (54) - : : :- CometProject (46) - : : : +- CometFilter (45) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : +- CometProject (53) - : : +- CometBroadcastHashJoin (52) - : : :- CometBroadcastExchange (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : +- CometProject (51) - : : +- CometFilter (50) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : +- ReusedExchange (55) - +- CometBroadcastExchange (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) +* CometColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometHashAggregate (69) + +- CometExchange (68) + +- CometHashAggregate (67) + +- CometExpand (66) + +- CometUnion (65) + :- CometHashAggregate (22) + : +- CometExchange (21) + : +- CometHashAggregate (20) + : +- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometUnion (7) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : +- CometBroadcastExchange (11) + : : +- CometProject (10) + : : +- CometFilter (9) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) + : +- CometBroadcastExchange (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + :- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometUnion (29) + : : : :- CometProject (25) + : : : : +- CometFilter (24) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : +- ReusedExchange (30) + : +- CometBroadcastExchange (36) + : +- CometProject (35) + : +- CometFilter (34) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometUnion (52) + : : :- CometProject (44) + : : : +- CometFilter (43) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometBroadcastExchange (46) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : +- ReusedExchange (53) + +- CometBroadcastExchange (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -176,296 +173,280 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Keys [1]: [s_store_id#26] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#35, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#36, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#37, store channel AS channel#38, concat(store, s_store_id#26) AS id#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : isnotnull(cs_catalog_page_sk#31) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(25) CometProject +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Condition : isnotnull(cr_catalog_page_sk#42) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(28) CometProject +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(29) CometUnion +Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [1]: [d_date_sk#52] +Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight -(33) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(32) CometProject +Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] +Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#63, 16, true, false, true) AS cp_catalog_page_id#64] +(35) CometProject +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(36) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(37) CometBroadcastHashJoin +Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(38) CometProject +Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(39) CometHashAggregate +Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometExchange +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(41) CometHashAggregate +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#73, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#74, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#75, catalog channel AS channel#76, concat(catalog_page, cp_catalog_page_id#64) AS id#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(43) CometFilter +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_web_site_sk#60) -(46) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(44) CometProject +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(48) CometFilter +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(49) CometProject +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#103] +(51) CometProject +Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [1]: [d_date_sk#86] +Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(55) CometProject +Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] +Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#87, web_site_id#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(57) CometFilter +Input [2]: [web_site_sk#87, web_site_id#88] +Condition : isnotnull(web_site_sk#87) + +(58) CometProject +Input [2]: [web_site_sk#87, web_site_id#88] +Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] + +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] -(60) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#106, 16, true, false, true) AS web_site_id#107] +(60) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [2]: [web_site_sk#87, web_site_id#89] +Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight -(61) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(61) CometProject +Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] +Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -(62) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], Inner, BuildRight +(62) CometHashAggregate +Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] +Keys [1]: [web_site_id#89] +Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] -(63) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(63) CometExchange +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (64) CometHashAggregate -Input [5]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#116, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#117, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#118, web channel AS channel#119, concat(web_site, web_site_id#107) AS id#120] - -(68) Union - -(69) Expand [codegen id : 4] -Input [5]: [sales#35, returns#36, profit#37, channel#38, id#39] -Arguments: [[sales#35, returns#36, profit#37, channel#38, id#39, 0], [sales#35, returns#36, profit#37, channel#38, null, 1], [sales#35, returns#36, profit#37, null, null, 3]], [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] - -(70) HashAggregate [codegen id : 4] -Input [6]: [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [partial_sum(sales#35), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Results [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(71) CometColumnarExchange -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Arguments: hashpartitioning(channel#121, id#122, spark_grouping_id#123, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(72) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(73) HashAggregate [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [sum(sales#35), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#35)#136, sum(returns#36)#137, sum(profit#37)#138] -Results [5]: [channel#121, id#122, sum(sales#35)#136 AS sales#139, sum(returns#36)#137 AS returns#140, sum(profit#37)#138 AS profit#141] - -(74) TakeOrderedAndProject -Input [5]: [channel#121, id#122, sales#139, returns#140, profit#141] -Arguments: 100, [channel#121 ASC NULLS FIRST, id#122 ASC NULLS FIRST], [channel#121, id#122, sales#139, returns#140, profit#141] +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Keys [1]: [web_site_id#89] +Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] + +(65) CometUnion +Child 0 Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] +Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] +Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] + +(66) CometExpand +Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] +Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] + +(67) CometHashAggregate +Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] + +(68) CometExchange +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(69) CometHashAggregate +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] + +(70) CometTakeOrderedAndProject +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] + +(71) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter +(73) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(77) CometProject +(74) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(79) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/simplified.txt index e86e0869f5..6fff89a25c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5.native_iceberg_compat/simplified.txt @@ -1,97 +1,86 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [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_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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #4 - 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 [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] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #8 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [web_site_sk,web_site_id] #10 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] + CometExchange [s_store_id] #2 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [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_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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #4 + 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 [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] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #6 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] + CometExchange [web_site_id] #8 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt index f2dce3f729..b4a1dedd61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/explain.txt @@ -1,78 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Expand (69) - +- Union (68) - :- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- * HashAggregate (43) - : +- * CometColumnarToRow (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (33) - : : +- CometBroadcastHashJoin (32) - : : :- CometUnion (30) - : : : :- CometProject (26) - : : : : +- CometFilter (25) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : +- ReusedExchange (31) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - +- * HashAggregate (67) - +- * CometColumnarToRow (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometProject (63) - +- CometBroadcastHashJoin (62) - :- CometProject (57) - : +- CometBroadcastHashJoin (56) - : :- CometUnion (54) - : : :- CometProject (46) - : : : +- CometFilter (45) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : +- CometProject (53) - : : +- CometBroadcastHashJoin (52) - : : :- CometBroadcastExchange (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : +- CometProject (51) - : : +- CometFilter (50) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : +- ReusedExchange (55) - +- CometBroadcastExchange (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) +* CometColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometHashAggregate (69) + +- CometExchange (68) + +- CometHashAggregate (67) + +- CometExpand (66) + +- CometUnion (65) + :- CometHashAggregate (22) + : +- CometExchange (21) + : +- CometHashAggregate (20) + : +- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometUnion (7) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : +- CometBroadcastExchange (11) + : : +- CometProject (10) + : : +- CometFilter (9) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) + : +- CometBroadcastExchange (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + :- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometUnion (29) + : : : :- CometProject (25) + : : : : +- CometFilter (24) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : +- ReusedExchange (30) + : +- CometBroadcastExchange (36) + : +- CometProject (35) + : +- CometFilter (34) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometUnion (52) + : : :- CometProject (44) + : : : +- CometFilter (43) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometBroadcastExchange (46) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : +- ReusedExchange (53) + +- CometBroadcastExchange (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -176,296 +173,280 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Keys [1]: [s_store_id#26] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#35, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#36, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#37, store channel AS channel#38, concat(store, s_store_id#26) AS id#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : isnotnull(cs_catalog_page_sk#31) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(25) CometProject +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Condition : isnotnull(cr_catalog_page_sk#42) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(28) CometProject +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(29) CometUnion +Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [1]: [d_date_sk#52] +Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight -(33) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(32) CometProject +Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] +Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#63, 16, true, false, true) AS cp_catalog_page_id#64] +(35) CometProject +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(36) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(37) CometBroadcastHashJoin +Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(38) CometProject +Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(39) CometHashAggregate +Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometExchange +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(41) CometHashAggregate +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#73, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#74, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#75, catalog channel AS channel#76, concat(catalog_page, cp_catalog_page_id#64) AS id#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(43) CometFilter +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_web_site_sk#60) -(46) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(44) CometProject +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(48) CometFilter +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(49) CometProject +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#103] +(51) CometProject +Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [1]: [d_date_sk#86] +Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(55) CometProject +Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] +Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#87, web_site_id#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(57) CometFilter +Input [2]: [web_site_sk#87, web_site_id#88] +Condition : isnotnull(web_site_sk#87) + +(58) CometProject +Input [2]: [web_site_sk#87, web_site_id#88] +Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] + +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] -(60) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#106, 16, true, false, true) AS web_site_id#107] +(60) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [2]: [web_site_sk#87, web_site_id#89] +Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight -(61) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(61) CometProject +Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] +Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -(62) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], Inner, BuildRight +(62) CometHashAggregate +Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] +Keys [1]: [web_site_id#89] +Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] -(63) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(63) CometExchange +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (64) CometHashAggregate -Input [5]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#116, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#117, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#118, web channel AS channel#119, concat(web_site, web_site_id#107) AS id#120] - -(68) Union - -(69) Expand [codegen id : 4] -Input [5]: [sales#35, returns#36, profit#37, channel#38, id#39] -Arguments: [[sales#35, returns#36, profit#37, channel#38, id#39, 0], [sales#35, returns#36, profit#37, channel#38, null, 1], [sales#35, returns#36, profit#37, null, null, 3]], [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] - -(70) HashAggregate [codegen id : 4] -Input [6]: [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [partial_sum(sales#35), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Results [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(71) CometColumnarExchange -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Arguments: hashpartitioning(channel#121, id#122, spark_grouping_id#123, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(72) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(73) HashAggregate [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [sum(sales#35), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#35)#136, sum(returns#36)#137, sum(profit#37)#138] -Results [5]: [channel#121, id#122, sum(sales#35)#136 AS sales#139, sum(returns#36)#137 AS returns#140, sum(profit#37)#138 AS profit#141] - -(74) TakeOrderedAndProject -Input [5]: [channel#121, id#122, sales#139, returns#140, profit#141] -Arguments: 100, [channel#121 ASC NULLS FIRST, id#122 ASC NULLS FIRST], [channel#121, id#122, sales#139, returns#140, profit#141] +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Keys [1]: [web_site_id#89] +Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] + +(65) CometUnion +Child 0 Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] +Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] +Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] + +(66) CometExpand +Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] +Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] + +(67) CometHashAggregate +Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] + +(68) CometExchange +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(69) CometHashAggregate +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] + +(70) CometTakeOrderedAndProject +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] + +(71) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter +(73) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(77) CometProject +(74) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(79) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt index e86e0869f5..6fff89a25c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q5/simplified.txt @@ -1,97 +1,86 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [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_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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #4 - 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 [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] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #8 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [web_site_sk,web_site_id] #10 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] + CometExchange [s_store_id] #2 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [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_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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #4 + 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 [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] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #6 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] + CometExchange [web_site_id] #8 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/explain.txt index bbc01dd8aa..f0a163bf69 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/explain.txt @@ -1,110 +1,107 @@ == Physical Plan == -TakeOrderedAndProject (106) -+- * HashAggregate (105) - +- * CometColumnarToRow (104) - +- CometColumnarExchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * HashAggregate (39) - : +- * CometColumnarToRow (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometProject (17) - : : : : +- CometBroadcastHashJoin (16) - : : : : :- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : +- CometBroadcastExchange (15) - : : : : +- CometProject (14) - : : : : +- CometFilter (13) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : +- CometBroadcastExchange (21) - : : : +- CometProject (20) - : : : +- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - :- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (53) - : : : : +- CometBroadcastHashJoin (52) - : : : : :- CometProject (50) - : : : : : +- CometSortMergeJoin (49) - : : : : : :- CometSort (43) - : : : : : : +- CometExchange (42) - : : : : : : +- CometFilter (41) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : +- CometSort (48) - : : : : : +- CometExchange (47) - : : : : : +- CometProject (46) - : : : : : +- CometFilter (45) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : +- ReusedExchange (51) - : : : +- CometBroadcastExchange (57) - : : : +- CometProject (56) - : : : +- CometFilter (55) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- * HashAggregate (99) - +- * CometColumnarToRow (98) - +- CometExchange (97) - +- CometHashAggregate (96) - +- CometProject (95) - +- CometBroadcastHashJoin (94) - :- CometProject (92) - : +- CometBroadcastHashJoin (91) - : :- CometProject (89) - : : +- CometBroadcastHashJoin (88) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (80) - : : : : +- CometSortMergeJoin (79) - : : : : :- CometSort (73) - : : : : : +- CometExchange (72) - : : : : : +- CometFilter (71) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : +- CometSort (78) - : : : : +- CometExchange (77) - : : : : +- CometProject (76) - : : : : +- CometFilter (75) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : +- ReusedExchange (81) - : : +- CometBroadcastExchange (87) - : : +- CometProject (86) - : : +- CometFilter (85) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : +- ReusedExchange (90) - +- ReusedExchange (93) +* CometColumnarToRow (103) ++- CometTakeOrderedAndProject (102) + +- CometHashAggregate (101) + +- CometExchange (100) + +- CometHashAggregate (99) + +- CometExpand (98) + +- CometUnion (97) + :- CometHashAggregate (38) + : +- CometExchange (37) + : +- CometHashAggregate (36) + : +- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometProject (17) + : : : : +- CometBroadcastHashJoin (16) + : : : : :- CometProject (11) + : : : : : +- CometSortMergeJoin (10) + : : : : : :- CometSort (4) + : : : : : : +- CometExchange (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : +- CometSort (9) + : : : : : +- CometExchange (8) + : : : : : +- CometProject (7) + : : : : : +- CometFilter (6) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : +- CometBroadcastExchange (15) + : : : : +- CometProject (14) + : : : : +- CometFilter (13) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) + : : : +- CometBroadcastExchange (21) + : : : +- CometProject (20) + : : : +- CometFilter (19) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) + : : +- CometBroadcastExchange (27) + : : +- CometProject (26) + : : +- CometFilter (25) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + : +- CometBroadcastExchange (33) + : +- CometProject (32) + : +- CometFilter (31) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + :- CometHashAggregate (67) + : +- CometExchange (66) + : +- CometHashAggregate (65) + : +- CometProject (64) + : +- CometBroadcastHashJoin (63) + : :- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (58) + : : : +- CometBroadcastHashJoin (57) + : : : :- CometProject (52) + : : : : +- CometBroadcastHashJoin (51) + : : : : :- CometProject (49) + : : : : : +- CometSortMergeJoin (48) + : : : : : :- CometSort (42) + : : : : : : +- CometExchange (41) + : : : : : : +- CometFilter (40) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) + : : : : : +- CometSort (47) + : : : : : +- CometExchange (46) + : : : : : +- CometProject (45) + : : : : : +- CometFilter (44) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : : : +- ReusedExchange (50) + : : : +- CometBroadcastExchange (56) + : : : +- CometProject (55) + : : : +- CometFilter (54) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) + : : +- ReusedExchange (59) + : +- ReusedExchange (62) + +- CometHashAggregate (96) + +- CometExchange (95) + +- CometHashAggregate (94) + +- CometProject (93) + +- CometBroadcastHashJoin (92) + :- CometProject (90) + : +- CometBroadcastHashJoin (89) + : :- CometProject (87) + : : +- CometBroadcastHashJoin (86) + : : :- CometProject (81) + : : : +- CometBroadcastHashJoin (80) + : : : :- CometProject (78) + : : : : +- CometSortMergeJoin (77) + : : : : :- CometSort (71) + : : : : : +- CometExchange (70) + : : : : : +- CometFilter (69) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) + : : : : +- CometSort (76) + : : : : +- CometExchange (75) + : : : : +- CometProject (74) + : : : : +- CometFilter (73) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) + : : : +- ReusedExchange (79) + : : +- CometBroadcastExchange (85) + : : +- CometProject (84) + : : +- CometFilter (83) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) + : +- ReusedExchange (88) + +- ReusedExchange (91) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -280,351 +277,335 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(38) CometHashAggregate Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Keys [1]: [s_store_id#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#33, store channel AS channel#34, concat(store, s_store_id#18) AS id#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) -(42) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(41) CometExchange +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 ASC NULLS FIRST] +(42) CometSort +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(44) CometFilter +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) -(46) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(45) CometProject +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(46) CometExchange +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 ASC NULLS FIRST] +(47) CometSort +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], LeftOuter +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(49) CometProject +Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(52) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#51, 16, true, false, true) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(55) CometProject +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(56) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(57) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(58) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(60) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [i_item_sk#45] +Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(61) CometProject +Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] +Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(63) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [p_promo_sk#46] +Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(64) CometProject +Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] +Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(65) CometHashAggregate +Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(66) CometExchange +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#63, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#64, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#65, catalog channel AS channel#66, concat(catalog_page, cp_catalog_page_id#52) AS id#67] +(67) CometHashAggregate +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(69) CometFilter +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(70) CometExchange +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 ASC NULLS FIRST] +(71) CometSort +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(73) CometFilter +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(74) CometProject +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(75) CometExchange +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(76) CometSort +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], LeftOuter +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(78) CometProject +Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +Right output [1]: [d_date_sk#65] +Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(81) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#66, web_site_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#83, 16, true, false, true) AS web_site_id#84] +(84) CometProject +Input [2]: [web_site_sk#66, web_site_id#67] +Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +Right output [2]: [web_site_sk#66, web_site_id#68] +Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(87) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] +Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [i_item_sk#69] +Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(90) CometProject +Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] +Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(92) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [p_promo_sk#70] +Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(93) CometProject +Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] +Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(94) CometHashAggregate +Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Keys [1]: [web_site_id#68] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(95) CometExchange +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] (96) CometHashAggregate -Input [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#95, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#96, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#97, web channel AS channel#98, concat(web_site, web_site_id#84) AS id#99] - -(100) Union - -(101) Expand [codegen id : 4] -Input [5]: [sales#31, returns#32, profit#33, channel#34, id#35] -Arguments: [[sales#31, returns#32, profit#33, channel#34, id#35, 0], [sales#31, returns#32, profit#33, channel#34, null, 1], [sales#31, returns#32, profit#33, null, null, 3]], [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] - -(102) HashAggregate [codegen id : 4] -Input [6]: [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [partial_sum(sales#31), partial_sum(returns#32), partial_sum(profit#33)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(103) CometColumnarExchange -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#100, id#101, spark_grouping_id#102, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(104) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(105) HashAggregate [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [sum(sales#31), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#31)#115, sum(returns#32)#116, sum(profit#33)#117] -Results [5]: [channel#100, id#101, sum(sales#31)#115 AS sales#118, sum(returns#32)#116 AS returns#119, sum(profit#33)#117 AS profit#120] - -(106) TakeOrderedAndProject -Input [5]: [channel#100, id#101, sales#118, returns#119, profit#120] -Arguments: 100, [channel#100 ASC NULLS FIRST, id#101 ASC NULLS FIRST], [channel#100, id#101, sales#118, returns#119, profit#120] +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Keys [1]: [web_site_id#68] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(97) CometUnion +Child 0 Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] +Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] +Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] + +(98) CometExpand +Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] +Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] + +(99) CometHashAggregate +Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] + +(100) CometExchange +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(101) CometHashAggregate +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] + +(102) CometTakeOrderedAndProject +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] + +(103) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometProject (109) - +- CometFilter (108) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (107) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(104) 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,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] ReadSchema: struct -(108) CometFilter +(105) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(109) CometProject +(106) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(110) CometColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(111) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/simplified.txt index 2c56b98787..198646e0fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80.native_iceberg_compat/simplified.txt @@ -1,126 +1,115 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - 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] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - 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] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] + CometExchange [s_store_id] #2 + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #5 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + 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] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + 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] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #9 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #10 + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #11 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] + CometExchange [web_site_id] #14 + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #15 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #16 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt index bbc01dd8aa..f0a163bf69 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/explain.txt @@ -1,110 +1,107 @@ == Physical Plan == -TakeOrderedAndProject (106) -+- * HashAggregate (105) - +- * CometColumnarToRow (104) - +- CometColumnarExchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * HashAggregate (39) - : +- * CometColumnarToRow (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometProject (17) - : : : : +- CometBroadcastHashJoin (16) - : : : : :- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : +- CometBroadcastExchange (15) - : : : : +- CometProject (14) - : : : : +- CometFilter (13) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : +- CometBroadcastExchange (21) - : : : +- CometProject (20) - : : : +- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - :- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (53) - : : : : +- CometBroadcastHashJoin (52) - : : : : :- CometProject (50) - : : : : : +- CometSortMergeJoin (49) - : : : : : :- CometSort (43) - : : : : : : +- CometExchange (42) - : : : : : : +- CometFilter (41) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : +- CometSort (48) - : : : : : +- CometExchange (47) - : : : : : +- CometProject (46) - : : : : : +- CometFilter (45) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : +- ReusedExchange (51) - : : : +- CometBroadcastExchange (57) - : : : +- CometProject (56) - : : : +- CometFilter (55) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- * HashAggregate (99) - +- * CometColumnarToRow (98) - +- CometExchange (97) - +- CometHashAggregate (96) - +- CometProject (95) - +- CometBroadcastHashJoin (94) - :- CometProject (92) - : +- CometBroadcastHashJoin (91) - : :- CometProject (89) - : : +- CometBroadcastHashJoin (88) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (80) - : : : : +- CometSortMergeJoin (79) - : : : : :- CometSort (73) - : : : : : +- CometExchange (72) - : : : : : +- CometFilter (71) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : +- CometSort (78) - : : : : +- CometExchange (77) - : : : : +- CometProject (76) - : : : : +- CometFilter (75) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : +- ReusedExchange (81) - : : +- CometBroadcastExchange (87) - : : +- CometProject (86) - : : +- CometFilter (85) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : +- ReusedExchange (90) - +- ReusedExchange (93) +* CometColumnarToRow (103) ++- CometTakeOrderedAndProject (102) + +- CometHashAggregate (101) + +- CometExchange (100) + +- CometHashAggregate (99) + +- CometExpand (98) + +- CometUnion (97) + :- CometHashAggregate (38) + : +- CometExchange (37) + : +- CometHashAggregate (36) + : +- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometProject (17) + : : : : +- CometBroadcastHashJoin (16) + : : : : :- CometProject (11) + : : : : : +- CometSortMergeJoin (10) + : : : : : :- CometSort (4) + : : : : : : +- CometExchange (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : +- CometSort (9) + : : : : : +- CometExchange (8) + : : : : : +- CometProject (7) + : : : : : +- CometFilter (6) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : +- CometBroadcastExchange (15) + : : : : +- CometProject (14) + : : : : +- CometFilter (13) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) + : : : +- CometBroadcastExchange (21) + : : : +- CometProject (20) + : : : +- CometFilter (19) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) + : : +- CometBroadcastExchange (27) + : : +- CometProject (26) + : : +- CometFilter (25) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + : +- CometBroadcastExchange (33) + : +- CometProject (32) + : +- CometFilter (31) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + :- CometHashAggregate (67) + : +- CometExchange (66) + : +- CometHashAggregate (65) + : +- CometProject (64) + : +- CometBroadcastHashJoin (63) + : :- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (58) + : : : +- CometBroadcastHashJoin (57) + : : : :- CometProject (52) + : : : : +- CometBroadcastHashJoin (51) + : : : : :- CometProject (49) + : : : : : +- CometSortMergeJoin (48) + : : : : : :- CometSort (42) + : : : : : : +- CometExchange (41) + : : : : : : +- CometFilter (40) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) + : : : : : +- CometSort (47) + : : : : : +- CometExchange (46) + : : : : : +- CometProject (45) + : : : : : +- CometFilter (44) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : : : +- ReusedExchange (50) + : : : +- CometBroadcastExchange (56) + : : : +- CometProject (55) + : : : +- CometFilter (54) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) + : : +- ReusedExchange (59) + : +- ReusedExchange (62) + +- CometHashAggregate (96) + +- CometExchange (95) + +- CometHashAggregate (94) + +- CometProject (93) + +- CometBroadcastHashJoin (92) + :- CometProject (90) + : +- CometBroadcastHashJoin (89) + : :- CometProject (87) + : : +- CometBroadcastHashJoin (86) + : : :- CometProject (81) + : : : +- CometBroadcastHashJoin (80) + : : : :- CometProject (78) + : : : : +- CometSortMergeJoin (77) + : : : : :- CometSort (71) + : : : : : +- CometExchange (70) + : : : : : +- CometFilter (69) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) + : : : : +- CometSort (76) + : : : : +- CometExchange (75) + : : : : +- CometProject (74) + : : : : +- CometFilter (73) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) + : : : +- ReusedExchange (79) + : : +- CometBroadcastExchange (85) + : : +- CometProject (84) + : : +- CometFilter (83) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) + : +- ReusedExchange (88) + +- ReusedExchange (91) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -280,351 +277,335 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(38) CometHashAggregate Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Keys [1]: [s_store_id#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#33, store channel AS channel#34, concat(store, s_store_id#18) AS id#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) -(42) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(41) CometExchange +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 ASC NULLS FIRST] +(42) CometSort +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(44) CometFilter +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) -(46) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(45) CometProject +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(46) CometExchange +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 ASC NULLS FIRST] +(47) CometSort +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], LeftOuter +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(49) CometProject +Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(52) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#51, 16, true, false, true) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(55) CometProject +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(56) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(57) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(58) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(60) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [i_item_sk#45] +Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(61) CometProject +Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] +Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(63) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [p_promo_sk#46] +Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(64) CometProject +Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] +Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(65) CometHashAggregate +Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(66) CometExchange +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#63, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#64, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#65, catalog channel AS channel#66, concat(catalog_page, cp_catalog_page_id#52) AS id#67] +(67) CometHashAggregate +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(69) CometFilter +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(70) CometExchange +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 ASC NULLS FIRST] +(71) CometSort +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(73) CometFilter +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(74) CometProject +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(75) CometExchange +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(76) CometSort +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], LeftOuter +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(78) CometProject +Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +Right output [1]: [d_date_sk#65] +Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(81) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#66, web_site_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#83, 16, true, false, true) AS web_site_id#84] +(84) CometProject +Input [2]: [web_site_sk#66, web_site_id#67] +Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +Right output [2]: [web_site_sk#66, web_site_id#68] +Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(87) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] +Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [i_item_sk#69] +Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(90) CometProject +Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] +Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(92) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [p_promo_sk#70] +Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(93) CometProject +Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] +Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(94) CometHashAggregate +Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Keys [1]: [web_site_id#68] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(95) CometExchange +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] (96) CometHashAggregate -Input [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#95, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#96, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#97, web channel AS channel#98, concat(web_site, web_site_id#84) AS id#99] - -(100) Union - -(101) Expand [codegen id : 4] -Input [5]: [sales#31, returns#32, profit#33, channel#34, id#35] -Arguments: [[sales#31, returns#32, profit#33, channel#34, id#35, 0], [sales#31, returns#32, profit#33, channel#34, null, 1], [sales#31, returns#32, profit#33, null, null, 3]], [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] - -(102) HashAggregate [codegen id : 4] -Input [6]: [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [partial_sum(sales#31), partial_sum(returns#32), partial_sum(profit#33)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(103) CometColumnarExchange -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#100, id#101, spark_grouping_id#102, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(104) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(105) HashAggregate [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [sum(sales#31), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#31)#115, sum(returns#32)#116, sum(profit#33)#117] -Results [5]: [channel#100, id#101, sum(sales#31)#115 AS sales#118, sum(returns#32)#116 AS returns#119, sum(profit#33)#117 AS profit#120] - -(106) TakeOrderedAndProject -Input [5]: [channel#100, id#101, sales#118, returns#119, profit#120] -Arguments: 100, [channel#100 ASC NULLS FIRST, id#101 ASC NULLS FIRST], [channel#100, id#101, sales#118, returns#119, profit#120] +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Keys [1]: [web_site_id#68] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(97) CometUnion +Child 0 Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] +Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] +Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] + +(98) CometExpand +Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] +Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] + +(99) CometHashAggregate +Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] + +(100) CometExchange +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(101) CometHashAggregate +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] + +(102) CometTakeOrderedAndProject +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] + +(103) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometProject (109) - +- CometFilter (108) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (107) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(104) 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,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] ReadSchema: struct -(108) CometFilter +(105) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(109) CometProject +(106) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(110) CometColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(111) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt index 2c56b98787..198646e0fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q80/simplified.txt @@ -1,126 +1,115 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - 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] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - 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] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] + CometExchange [s_store_id] #2 + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #5 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + 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] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + 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] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #9 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #10 + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #11 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] + CometExchange [web_site_id] #14 + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #15 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #16 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/explain.txt index 45c375a787..5bafdc5586 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * Project (32) - +- * CometColumnarToRow (31) +* CometColumnarToRow (33) ++- CometTakeOrderedAndProject (32) + +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometBroadcastExchange (26) : +- CometProject (25) @@ -177,14 +177,14 @@ Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] Right output [1]: [sr_cdemo_sk#18] Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft -(31) CometColumnarToRow [codegen id : 1] +(31) CometProject Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] +Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -(32) Project [codegen id : 1] -Output [3]: [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] - -(33) TakeOrderedAndProject +(32) CometTakeOrderedAndProject Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: 100, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] + +(33) CometColumnarToRow [codegen id : 1] +Input [2]: [customer_id#20, customername#21] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/simplified.txt index cc7cf901e5..fd8d1864ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [c_customer_id,customer_id,customername] - WholeStageCodegen (1) - Project [c_customer_id,c_last_name,c_first_name] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customername,c_customer_id] + CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt index 45c375a787..5bafdc5586 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * Project (32) - +- * CometColumnarToRow (31) +* CometColumnarToRow (33) ++- CometTakeOrderedAndProject (32) + +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometBroadcastExchange (26) : +- CometProject (25) @@ -177,14 +177,14 @@ Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] Right output [1]: [sr_cdemo_sk#18] Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft -(31) CometColumnarToRow [codegen id : 1] +(31) CometProject Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] +Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -(32) Project [codegen id : 1] -Output [3]: [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] - -(33) TakeOrderedAndProject +(32) CometTakeOrderedAndProject Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: 100, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] + +(33) CometColumnarToRow [codegen id : 1] +Input [2]: [customer_id#20, customername#21] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt index cc7cf901e5..fd8d1864ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark3_5/q84/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [c_customer_id,customer_id,customername] - WholeStageCodegen (1) - Project [c_customer_id,c_last_name,c_first_name] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customername,c_customer_id] + CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] From 73d2a011f3c8074629e616e5ff9e08a8b14c6ce3 Mon Sep 17 00:00:00 2001 From: comphead Date: Mon, 27 Oct 2025 09:39:37 -0700 Subject: [PATCH 6/8] feat: support `concat` for strings --- .../q5a.native_iceberg_compat/explain.txt | 664 +++++++------- .../q5a.native_iceberg_compat/simplified.txt | 218 ++--- .../q5a/explain.txt | 664 +++++++------- .../q5a/simplified.txt | 218 ++--- .../q80a.native_iceberg_compat/explain.txt | 820 ++++++++---------- .../q80a.native_iceberg_compat/simplified.txt | 270 +++--- .../q80a/explain.txt | 820 ++++++++---------- .../q80a/simplified.txt | 270 +++--- 8 files changed, 1816 insertions(+), 2128 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt index 162121609a..33c3a5de2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/explain.txt @@ -1,96 +1,88 @@ == Physical Plan == -* CometColumnarToRow (92) -+- CometTakeOrderedAndProject (91) - +- CometHashAggregate (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- Union (87) - :- * HashAggregate (72) - : +- * CometColumnarToRow (71) - : +- CometColumnarExchange (70) - : +- * HashAggregate (69) - : +- Union (68) - : :- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- * HashAggregate (43) - : : +- * CometColumnarToRow (42) - : : +- CometExchange (41) - : : +- CometHashAggregate (40) - : : +- CometProject (39) - : : +- CometBroadcastHashJoin (38) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometUnion (30) - : : : : :- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : : +- CometProject (29) - : : : : +- CometFilter (28) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : : +- ReusedExchange (31) - : : +- CometBroadcastExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometUnion (54) - : : : :- CometProject (46) - : : : : +- CometFilter (45) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : : +- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastExchange (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : : +- CometProject (51) - : : : +- CometFilter (50) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : : +- ReusedExchange (55) - : +- CometBroadcastExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) - :- * HashAggregate (79) - : +- * CometColumnarToRow (78) - : +- CometColumnarExchange (77) - : +- * HashAggregate (76) - : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) - : +- ReusedExchange (73) - +- * HashAggregate (86) - +- * CometColumnarToRow (85) - +- CometColumnarExchange (84) - +- * HashAggregate (83) - +- * HashAggregate (82) - +- * CometColumnarToRow (81) - +- ReusedExchange (80) +* CometColumnarToRow (84) ++- CometTakeOrderedAndProject (83) + +- CometHashAggregate (82) + +- CometExchange (81) + +- CometHashAggregate (80) + +- CometUnion (79) + :- CometHashAggregate (68) + : +- CometExchange (67) + : +- CometHashAggregate (66) + : +- CometUnion (65) + : :- CometHashAggregate (22) + : : +- CometExchange (21) + : : +- CometHashAggregate (20) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometProject (10) + : : : +- CometFilter (9) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (17) + : : +- CometProject (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : :- CometHashAggregate (41) + : : +- CometExchange (40) + : : +- CometHashAggregate (39) + : : +- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (30) + : : +- CometBroadcastExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) + : +- CometHashAggregate (64) + : +- CometExchange (63) + : +- CometHashAggregate (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometUnion (52) + : : : :- CometProject (44) + : : : : +- CometFilter (43) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : : +- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometBroadcastExchange (46) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) + : : : +- CometProject (49) + : : : +- CometFilter (48) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (53) + : +- CometBroadcastExchange (59) + : +- CometProject (58) + : +- CometFilter (57) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) + :- CometHashAggregate (73) + : +- CometExchange (72) + : +- CometHashAggregate (71) + : +- CometHashAggregate (70) + : +- ReusedExchange (69) + +- CometHashAggregate (78) + +- CometExchange (77) + +- CometHashAggregate (76) + +- CometHashAggregate (75) + +- ReusedExchange (74) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -194,381 +186,339 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Keys [1]: [s_store_id#26] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [store channel AS channel#35, concat(store, s_store_id#26) AS id#36, MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : isnotnull(cs_catalog_page_sk#31) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(25) CometProject +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Condition : isnotnull(cr_catalog_page_sk#42) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(28) CometProject +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(29) CometUnion +Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [1]: [d_date_sk#52] +Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight -(33) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(32) CometProject +Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] +Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#63, 16, true, false, true) AS cp_catalog_page_id#64] +(35) CometProject +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(36) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(37) CometBroadcastHashJoin +Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(38) CometProject +Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(39) CometHashAggregate +Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometExchange +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(41) CometHashAggregate +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#64) AS id#74, MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#75, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#76, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(43) CometFilter +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_web_site_sk#60) -(46) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(44) CometProject +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(48) CometFilter +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(49) CometProject +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#103] +(51) CometProject +Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [1]: [d_date_sk#86] +Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(55) CometProject +Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] +Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#87, web_site_id#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(57) CometFilter +Input [2]: [web_site_sk#87, web_site_id#88] +Condition : isnotnull(web_site_sk#87) + +(58) CometProject +Input [2]: [web_site_sk#87, web_site_id#88] +Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] -(60) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#106, 16, true, false, true) AS web_site_id#107] +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] -(61) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(60) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [2]: [web_site_sk#87, web_site_id#89] +Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight -(62) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], Inner, BuildRight +(61) CometProject +Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] +Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -(63) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(62) CometHashAggregate +Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] +Keys [1]: [web_site_id#89] +Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] + +(63) CometExchange +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (64) CometHashAggregate -Input [5]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [web channel AS channel#116, concat(web_site, web_site_id#107) AS id#117, MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#118, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#119, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#120] - -(68) Union - -(69) HashAggregate [codegen id : 4] -Input [5]: [channel#35, id#36, sales#37, returns#38, profit#39] -Keys [2]: [channel#35, id#36] -Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Results [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(70) CometColumnarExchange -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: hashpartitioning(channel#35, id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(71) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(72) HashAggregate [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [5]: [channel#35, id#36, cast(sum(sales#37)#133 as decimal(37,2)) AS sales#136, cast(sum(returns#38)#134 as decimal(37,2)) AS returns#137, cast(sum(profit#39)#135 as decimal(38,2)) AS profit#138] - -(73) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(74) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(75) HashAggregate [codegen id : 10] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [4]: [channel#35, sum(sales#37)#133 AS sales#139, sum(returns#38)#134 AS returns#140, sum(profit#39)#135 AS profit#141] - -(76) HashAggregate [codegen id : 10] -Input [4]: [channel#35, sales#139, returns#140, profit#141] -Keys [1]: [channel#35] -Functions [3]: [partial_sum(sales#139), partial_sum(returns#140), partial_sum(profit#141)] -Aggregate Attributes [6]: [sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Results [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] - -(77) CometColumnarExchange -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Arguments: hashpartitioning(channel#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(78) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] - -(79) HashAggregate [codegen id : 11] -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Keys [1]: [channel#35] -Functions [3]: [sum(sales#139), sum(returns#140), sum(profit#141)] -Aggregate Attributes [3]: [sum(sales#139)#154, sum(returns#140)#155, sum(profit#141)#156] -Results [5]: [channel#35, null AS id#157, sum(sales#139)#154 AS sum(sales)#158, sum(returns#140)#155 AS sum(returns)#159, sum(profit#141)#156 AS sum(profit)#160] - -(80) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(81) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(82) HashAggregate [codegen id : 16] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [3]: [sum(sales#37)#133 AS sales#161, sum(returns#38)#134 AS returns#162, sum(profit#39)#135 AS profit#163] - -(83) HashAggregate [codegen id : 16] -Input [3]: [sales#161, returns#162, profit#163] +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Keys [1]: [web_site_id#89] +Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] + +(65) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] +Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] + +(66) CometHashAggregate +Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Keys [2]: [channel#94, id#95] +Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] + +(67) CometExchange +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(68) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(69) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(70) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(71) CometHashAggregate +Input [4]: [channel#94, sales#115, returns#116, profit#117] +Keys [1]: [channel#94] +Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] + +(72) CometExchange +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(73) CometHashAggregate +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Keys [1]: [channel#94] +Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] + +(74) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(75) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(76) CometHashAggregate +Input [3]: [sales#124, returns#125, profit#126] Keys: [] -Functions [3]: [partial_sum(sales#161), partial_sum(returns#162), partial_sum(profit#163)] -Aggregate Attributes [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] -Results [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] - -(84) CometColumnarExchange -Input [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] -(85) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +(77) CometExchange +Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(86) HashAggregate [codegen id : 17] -Input [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +(78) CometHashAggregate +Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] Keys: [] -Functions [3]: [sum(sales#161), sum(returns#162), sum(profit#163)] -Aggregate Attributes [3]: [sum(sales#161)#176, sum(returns#162)#177, sum(profit#163)#178] -Results [5]: [null AS channel#179, null AS id#180, sum(sales#161)#176 AS sum(sales)#181, sum(returns#162)#177 AS sum(returns)#182, sum(profit#163)#178 AS sum(profit)#183] +Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] -(87) Union +(79) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Child 1 Input [5]: [channel#94, id#136, sum(sales)#137, sum(returns)#138, sum(profit)#139] +Child 2 Input [5]: [channel#140, id#141, sum(sales)#142, sum(returns)#143, sum(profit)#144] -(88) HashAggregate [codegen id : 18] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(80) CometHashAggregate +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -(89) CometColumnarExchange -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: hashpartitioning(channel#35, id#36, sales#136, returns#137, profit#138, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(81) CometExchange +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Arguments: hashpartitioning(channel#94, id#95, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(90) CometHashAggregate -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(82) CometHashAggregate +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] Functions: [] -(91) CometTakeOrderedAndProject -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,id#36 ASC NULLS FIRST], output=[channel#35,id#36,sales#136,returns#137,profit#138]), [channel#35, id#36, sales#136, returns#137, profit#138], 100, 0, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35, id#36, sales#136, returns#137, profit#138] +(83) CometTakeOrderedAndProject +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#133,returns#134,profit#135]), [channel#94, id#95, sales#133, returns#134, profit#135], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#133, returns#134, profit#135] -(92) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(84) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (97) -+- * CometColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (93) +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(94) CometFilter +(86) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(95) CometProject +(87) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(96) CometColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(97) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt index f97451c0c9..c0b236a57a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a.native_iceberg_compat/simplified.txt @@ -1,127 +1,99 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [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_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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - 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] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - 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] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (11) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #12 - WholeStageCodegen (10) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (16) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [s_store_id] #3 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [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_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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + 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] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + 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] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [cp_catalog_page_id] #7 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [web_site_id] #9 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #12 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange #13 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt index 162121609a..33c3a5de2e 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/explain.txt @@ -1,96 +1,88 @@ == Physical Plan == -* CometColumnarToRow (92) -+- CometTakeOrderedAndProject (91) - +- CometHashAggregate (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- Union (87) - :- * HashAggregate (72) - : +- * CometColumnarToRow (71) - : +- CometColumnarExchange (70) - : +- * HashAggregate (69) - : +- Union (68) - : :- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- * HashAggregate (43) - : : +- * CometColumnarToRow (42) - : : +- CometExchange (41) - : : +- CometHashAggregate (40) - : : +- CometProject (39) - : : +- CometBroadcastHashJoin (38) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometUnion (30) - : : : : :- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : : +- CometProject (29) - : : : : +- CometFilter (28) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : : +- ReusedExchange (31) - : : +- CometBroadcastExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometUnion (54) - : : : :- CometProject (46) - : : : : +- CometFilter (45) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : : +- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastExchange (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : : +- CometProject (51) - : : : +- CometFilter (50) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : : +- ReusedExchange (55) - : +- CometBroadcastExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) - :- * HashAggregate (79) - : +- * CometColumnarToRow (78) - : +- CometColumnarExchange (77) - : +- * HashAggregate (76) - : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) - : +- ReusedExchange (73) - +- * HashAggregate (86) - +- * CometColumnarToRow (85) - +- CometColumnarExchange (84) - +- * HashAggregate (83) - +- * HashAggregate (82) - +- * CometColumnarToRow (81) - +- ReusedExchange (80) +* CometColumnarToRow (84) ++- CometTakeOrderedAndProject (83) + +- CometHashAggregate (82) + +- CometExchange (81) + +- CometHashAggregate (80) + +- CometUnion (79) + :- CometHashAggregate (68) + : +- CometExchange (67) + : +- CometHashAggregate (66) + : +- CometUnion (65) + : :- CometHashAggregate (22) + : : +- CometExchange (21) + : : +- CometHashAggregate (20) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometProject (10) + : : : +- CometFilter (9) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (17) + : : +- CometProject (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : :- CometHashAggregate (41) + : : +- CometExchange (40) + : : +- CometHashAggregate (39) + : : +- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (30) + : : +- CometBroadcastExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) + : +- CometHashAggregate (64) + : +- CometExchange (63) + : +- CometHashAggregate (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometUnion (52) + : : : :- CometProject (44) + : : : : +- CometFilter (43) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : : +- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometBroadcastExchange (46) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) + : : : +- CometProject (49) + : : : +- CometFilter (48) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (53) + : +- CometBroadcastExchange (59) + : +- CometProject (58) + : +- CometFilter (57) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) + :- CometHashAggregate (73) + : +- CometExchange (72) + : +- CometHashAggregate (71) + : +- CometHashAggregate (70) + : +- ReusedExchange (69) + +- CometHashAggregate (78) + +- CometExchange (77) + +- CometHashAggregate (76) + +- CometHashAggregate (75) + +- ReusedExchange (74) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -194,381 +186,339 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Keys [1]: [s_store_id#26] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [store channel AS channel#35, concat(store, s_store_id#26) AS id#36, MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : isnotnull(cs_catalog_page_sk#31) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(25) CometProject +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Condition : isnotnull(cr_catalog_page_sk#42) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(28) CometProject +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(29) CometUnion +Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [1]: [d_date_sk#52] +Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight -(33) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(32) CometProject +Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] +Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#63, 16, true, false, true) AS cp_catalog_page_id#64] +(35) CometProject +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(36) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(37) CometBroadcastHashJoin +Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(38) CometProject +Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(39) CometHashAggregate +Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometExchange +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(41) CometHashAggregate +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#64) AS id#74, MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#75, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#76, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(43) CometFilter +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_web_site_sk#60) -(46) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(44) CometProject +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(48) CometFilter +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(49) CometProject +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#103] +(51) CometProject +Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [1]: [d_date_sk#86] +Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(55) CometProject +Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] +Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#87, web_site_id#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(57) CometFilter +Input [2]: [web_site_sk#87, web_site_id#88] +Condition : isnotnull(web_site_sk#87) + +(58) CometProject +Input [2]: [web_site_sk#87, web_site_id#88] +Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] -(60) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#106, 16, true, false, true) AS web_site_id#107] +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] -(61) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(60) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [2]: [web_site_sk#87, web_site_id#89] +Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight -(62) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], Inner, BuildRight +(61) CometProject +Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] +Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -(63) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(62) CometHashAggregate +Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] +Keys [1]: [web_site_id#89] +Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] + +(63) CometExchange +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (64) CometHashAggregate -Input [5]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [web channel AS channel#116, concat(web_site, web_site_id#107) AS id#117, MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#118, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#119, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#120] - -(68) Union - -(69) HashAggregate [codegen id : 4] -Input [5]: [channel#35, id#36, sales#37, returns#38, profit#39] -Keys [2]: [channel#35, id#36] -Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Results [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(70) CometColumnarExchange -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: hashpartitioning(channel#35, id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(71) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(72) HashAggregate [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [5]: [channel#35, id#36, cast(sum(sales#37)#133 as decimal(37,2)) AS sales#136, cast(sum(returns#38)#134 as decimal(37,2)) AS returns#137, cast(sum(profit#39)#135 as decimal(38,2)) AS profit#138] - -(73) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(74) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(75) HashAggregate [codegen id : 10] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [4]: [channel#35, sum(sales#37)#133 AS sales#139, sum(returns#38)#134 AS returns#140, sum(profit#39)#135 AS profit#141] - -(76) HashAggregate [codegen id : 10] -Input [4]: [channel#35, sales#139, returns#140, profit#141] -Keys [1]: [channel#35] -Functions [3]: [partial_sum(sales#139), partial_sum(returns#140), partial_sum(profit#141)] -Aggregate Attributes [6]: [sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Results [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] - -(77) CometColumnarExchange -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Arguments: hashpartitioning(channel#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(78) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] - -(79) HashAggregate [codegen id : 11] -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Keys [1]: [channel#35] -Functions [3]: [sum(sales#139), sum(returns#140), sum(profit#141)] -Aggregate Attributes [3]: [sum(sales#139)#154, sum(returns#140)#155, sum(profit#141)#156] -Results [5]: [channel#35, null AS id#157, sum(sales#139)#154 AS sum(sales)#158, sum(returns#140)#155 AS sum(returns)#159, sum(profit#141)#156 AS sum(profit)#160] - -(80) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(81) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(82) HashAggregate [codegen id : 16] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [3]: [sum(sales#37)#133 AS sales#161, sum(returns#38)#134 AS returns#162, sum(profit#39)#135 AS profit#163] - -(83) HashAggregate [codegen id : 16] -Input [3]: [sales#161, returns#162, profit#163] +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Keys [1]: [web_site_id#89] +Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] + +(65) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] +Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] + +(66) CometHashAggregate +Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Keys [2]: [channel#94, id#95] +Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] + +(67) CometExchange +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(68) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(69) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(70) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(71) CometHashAggregate +Input [4]: [channel#94, sales#115, returns#116, profit#117] +Keys [1]: [channel#94] +Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] + +(72) CometExchange +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(73) CometHashAggregate +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Keys [1]: [channel#94] +Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] + +(74) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(75) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(76) CometHashAggregate +Input [3]: [sales#124, returns#125, profit#126] Keys: [] -Functions [3]: [partial_sum(sales#161), partial_sum(returns#162), partial_sum(profit#163)] -Aggregate Attributes [6]: [sum#164, isEmpty#165, sum#166, isEmpty#167, sum#168, isEmpty#169] -Results [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] - -(84) CometColumnarExchange -Input [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Functions [3]: [partial_sum(sales#124), partial_sum(returns#125), partial_sum(profit#126)] -(85) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +(77) CometExchange +Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(86) HashAggregate [codegen id : 17] -Input [6]: [sum#170, isEmpty#171, sum#172, isEmpty#173, sum#174, isEmpty#175] +(78) CometHashAggregate +Input [6]: [sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] Keys: [] -Functions [3]: [sum(sales#161), sum(returns#162), sum(profit#163)] -Aggregate Attributes [3]: [sum(sales#161)#176, sum(returns#162)#177, sum(profit#163)#178] -Results [5]: [null AS channel#179, null AS id#180, sum(sales#161)#176 AS sum(sales)#181, sum(returns#162)#177 AS sum(returns)#182, sum(profit#163)#178 AS sum(profit)#183] +Functions [3]: [sum(sales#124), sum(returns#125), sum(profit#126)] -(87) Union +(79) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Child 1 Input [5]: [channel#94, id#136, sum(sales)#137, sum(returns)#138, sum(profit)#139] +Child 2 Input [5]: [channel#140, id#141, sum(sales)#142, sum(returns)#143, sum(profit)#144] -(88) HashAggregate [codegen id : 18] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(80) CometHashAggregate +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -(89) CometColumnarExchange -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: hashpartitioning(channel#35, id#36, sales#136, returns#137, profit#138, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(81) CometExchange +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Arguments: hashpartitioning(channel#94, id#95, sales#133, returns#134, profit#135, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(90) CometHashAggregate -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(82) CometHashAggregate +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Keys [5]: [channel#94, id#95, sales#133, returns#134, profit#135] Functions: [] -(91) CometTakeOrderedAndProject -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,id#36 ASC NULLS FIRST], output=[channel#35,id#36,sales#136,returns#137,profit#138]), [channel#35, id#36, sales#136, returns#137, profit#138], 100, 0, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35, id#36, sales#136, returns#137, profit#138] +(83) CometTakeOrderedAndProject +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#133,returns#134,profit#135]), [channel#94, id#95, sales#133, returns#134, profit#135], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#133, returns#134, profit#135] -(92) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(84) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#94, id#95, sales#133, returns#134, profit#135] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (97) -+- * CometColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (93) +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(94) CometFilter +(86) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(95) CometProject +(87) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(96) CometColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(97) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt index f97451c0c9..c0b236a57a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q5a/simplified.txt @@ -1,127 +1,99 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [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_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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - 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] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - 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] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (11) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #12 - WholeStageCodegen (10) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (16) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [s_store_id] #3 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [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_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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + 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] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + 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] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [cp_catalog_page_id] #7 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [web_site_id] #9 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #12 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange #13 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt index 7026380e53..f9901f13d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/explain.txt @@ -1,128 +1,120 @@ == Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometHashAggregate (122) - +- CometColumnarExchange (121) - +- * HashAggregate (120) - +- Union (119) - :- * HashAggregate (104) - : +- * CometColumnarToRow (103) - : +- CometColumnarExchange (102) - : +- * HashAggregate (101) - : +- Union (100) - : :- * HashAggregate (39) - : : +- * CometColumnarToRow (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) - : : +- CometExchange (67) - : : +- CometHashAggregate (66) - : : +- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (62) - : : : +- CometBroadcastHashJoin (61) - : : : :- CometProject (59) - : : : : +- CometBroadcastHashJoin (58) - : : : : :- CometProject (53) - : : : : : +- CometBroadcastHashJoin (52) - : : : : : :- CometProject (50) - : : : : : : +- CometSortMergeJoin (49) - : : : : : : :- CometSort (43) - : : : : : : : +- CometExchange (42) - : : : : : : : +- CometFilter (41) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- CometSort (48) - : : : : : : +- CometExchange (47) - : : : : : : +- CometProject (46) - : : : : : : +- CometFilter (45) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : : +- ReusedExchange (51) - : : : : +- CometBroadcastExchange (57) - : : : : +- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : : +- ReusedExchange (60) - : : +- ReusedExchange (63) - : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) - : +- CometExchange (97) - : +- CometHashAggregate (96) - : +- CometProject (95) - : +- CometBroadcastHashJoin (94) - : :- CometProject (92) - : : +- CometBroadcastHashJoin (91) - : : :- CometProject (89) - : : : +- CometBroadcastHashJoin (88) - : : : :- CometProject (83) - : : : : +- CometBroadcastHashJoin (82) - : : : : :- CometProject (80) - : : : : : +- CometSortMergeJoin (79) - : : : : : :- CometSort (73) - : : : : : : +- CometExchange (72) - : : : : : : +- CometFilter (71) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : : +- CometSort (78) - : : : : : +- CometExchange (77) - : : : : : +- CometProject (76) - : : : : : +- CometFilter (75) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : : +- ReusedExchange (81) - : : : +- CometBroadcastExchange (87) - : : : +- CometProject (86) - : : : +- CometFilter (85) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (111) - : +- * CometColumnarToRow (110) - : +- CometColumnarExchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * CometColumnarToRow (106) - : +- ReusedExchange (105) - +- * HashAggregate (118) - +- * CometColumnarToRow (117) - +- CometColumnarExchange (116) - +- * HashAggregate (115) - +- * HashAggregate (114) - +- * CometColumnarToRow (113) - +- ReusedExchange (112) +* CometColumnarToRow (116) ++- CometTakeOrderedAndProject (115) + +- CometHashAggregate (114) + +- CometExchange (113) + +- CometHashAggregate (112) + +- CometUnion (111) + :- CometHashAggregate (100) + : +- CometExchange (99) + : +- CometHashAggregate (98) + : +- CometUnion (97) + : :- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (29) + : : : +- CometBroadcastHashJoin (28) + : : : :- CometProject (23) + : : : : +- CometBroadcastHashJoin (22) + : : : : :- CometProject (17) + : : : : : +- CometBroadcastHashJoin (16) + : : : : : :- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : : +- CometBroadcastExchange (15) + : : : : : +- CometProject (14) + : : : : : +- CometFilter (13) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) + : : : : +- CometBroadcastExchange (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) + : : : +- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + : : +- CometBroadcastExchange (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + : :- CometHashAggregate (67) + : : +- CometExchange (66) + : : +- CometHashAggregate (65) + : : +- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometProject (61) + : : : +- CometBroadcastHashJoin (60) + : : : :- CometProject (58) + : : : : +- CometBroadcastHashJoin (57) + : : : : :- CometProject (52) + : : : : : +- CometBroadcastHashJoin (51) + : : : : : :- CometProject (49) + : : : : : : +- CometSortMergeJoin (48) + : : : : : : :- CometSort (42) + : : : : : : : +- CometExchange (41) + : : : : : : : +- CometFilter (40) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- CometSort (47) + : : : : : : +- CometExchange (46) + : : : : : : +- CometProject (45) + : : : : : : +- CometFilter (44) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : : : : +- ReusedExchange (50) + : : : : +- CometBroadcastExchange (56) + : : : : +- CometProject (55) + : : : : +- CometFilter (54) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) + : : : +- ReusedExchange (59) + : : +- ReusedExchange (62) + : +- CometHashAggregate (96) + : +- CometExchange (95) + : +- CometHashAggregate (94) + : +- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometProject (81) + : : : : +- CometBroadcastHashJoin (80) + : : : : :- CometProject (78) + : : : : : +- CometSortMergeJoin (77) + : : : : : :- CometSort (71) + : : : : : : +- CometExchange (70) + : : : : : : +- CometFilter (69) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) + : : : : : +- CometSort (76) + : : : : : +- CometExchange (75) + : : : : : +- CometProject (74) + : : : : : +- CometFilter (73) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) + : : : : +- ReusedExchange (79) + : : : +- CometBroadcastExchange (85) + : : : +- CometProject (84) + : : : +- CometFilter (83) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) + : : +- ReusedExchange (88) + : +- ReusedExchange (91) + :- CometHashAggregate (105) + : +- CometExchange (104) + : +- CometHashAggregate (103) + : +- CometHashAggregate (102) + : +- ReusedExchange (101) + +- CometHashAggregate (110) + +- CometExchange (109) + +- CometHashAggregate (108) + +- CometHashAggregate (107) + +- ReusedExchange (106) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -298,436 +290,394 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(38) CometHashAggregate Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Keys [1]: [s_store_id#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [store channel AS channel#31, concat(store, s_store_id#18) AS id#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#33, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#34, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) -(42) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(41) CometExchange +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 ASC NULLS FIRST] +(42) CometSort +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(44) CometFilter +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) -(46) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(45) CometProject +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(46) CometExchange +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 ASC NULLS FIRST] +(47) CometSort +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], LeftOuter +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(49) CometProject +Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(52) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#51, 16, true, false, true) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(55) CometProject +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(56) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(57) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(58) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(60) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [i_item_sk#45] +Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(61) CometProject +Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] +Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(63) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [p_promo_sk#46] +Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(64) CometProject +Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] +Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(65) CometHashAggregate +Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(66) CometExchange +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [catalog channel AS channel#63, concat(catalog_page, cp_catalog_page_id#52) AS id#64, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#65, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#66, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#67] +(67) CometHashAggregate +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(69) CometFilter +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(70) CometExchange +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 ASC NULLS FIRST] +(71) CometSort +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(73) CometFilter +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(74) CometProject +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(75) CometExchange +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(76) CometSort +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], LeftOuter +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(78) CometProject +Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +Right output [1]: [d_date_sk#65] +Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(81) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#66, web_site_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#83, 16, true, false, true) AS web_site_id#84] +(84) CometProject +Input [2]: [web_site_sk#66, web_site_id#67] +Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +Right output [2]: [web_site_sk#66, web_site_id#68] +Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(87) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] +Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [i_item_sk#69] +Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(90) CometProject +Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] +Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(92) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [p_promo_sk#70] +Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(93) CometProject +Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] +Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(94) CometHashAggregate +Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Keys [1]: [web_site_id#68] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(95) CometExchange +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] (96) CometHashAggregate -Input [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [web channel AS channel#95, concat(web_site, web_site_id#84) AS id#96, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#97, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#98, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#99] - -(100) Union - -(101) HashAggregate [codegen id : 4] -Input [5]: [channel#31, id#32, sales#33, returns#34, profit#35] -Keys [2]: [channel#31, id#32] -Functions [3]: [partial_sum(sales#33), partial_sum(returns#34), partial_sum(profit#35)] -Aggregate Attributes [6]: [sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Results [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(102) CometColumnarExchange -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(103) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(104) HashAggregate [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [5]: [channel#31, id#32, cast(sum(sales#33)#112 as decimal(37,2)) AS sales#115, cast(sum(returns#34)#113 as decimal(38,2)) AS returns#116, cast(sum(profit#35)#114 as decimal(38,2)) AS profit#117] - -(105) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(106) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(107) HashAggregate [codegen id : 10] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [4]: [channel#31, sum(sales#33)#112 AS sales#118, sum(returns#34)#113 AS returns#119, sum(profit#35)#114 AS profit#120] - -(108) HashAggregate [codegen id : 10] -Input [4]: [channel#31, sales#118, returns#119, profit#120] -Keys [1]: [channel#31] -Functions [3]: [partial_sum(sales#118), partial_sum(returns#119), partial_sum(profit#120)] -Aggregate Attributes [6]: [sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Results [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(109) CometColumnarExchange -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(110) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(111) HashAggregate [codegen id : 11] -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [1]: [channel#31] -Functions [3]: [sum(sales#118), sum(returns#119), sum(profit#120)] -Aggregate Attributes [3]: [sum(sales#118)#133, sum(returns#119)#134, sum(profit#120)#135] -Results [5]: [channel#31, null AS id#136, sum(sales#118)#133 AS sales#137, sum(returns#119)#134 AS returns#138, sum(profit#120)#135 AS profit#139] - -(112) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(113) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(114) HashAggregate [codegen id : 16] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [3]: [sum(sales#33)#112 AS sales#140, sum(returns#34)#113 AS returns#141, sum(profit#35)#114 AS profit#142] - -(115) HashAggregate [codegen id : 16] -Input [3]: [sales#140, returns#141, profit#142] +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Keys [1]: [web_site_id#68] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(97) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] +Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] + +(98) CometHashAggregate +Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Keys [2]: [channel#76, id#77] +Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] + +(99) CometExchange +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(100) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(101) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(102) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(103) CometHashAggregate +Input [4]: [channel#76, sales#97, returns#98, profit#99] +Keys [1]: [channel#76] +Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] + +(104) CometExchange +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(105) CometHashAggregate +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Keys [1]: [channel#76] +Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] + +(106) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(107) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(108) CometHashAggregate +Input [3]: [sales#106, returns#107, profit#108] Keys: [] -Functions [3]: [partial_sum(sales#140), partial_sum(returns#141), partial_sum(profit#142)] -Aggregate Attributes [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Results [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] - -(116) CometColumnarExchange -Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] -(117) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] +(109) CometExchange +Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(118) HashAggregate [codegen id : 17] -Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] +(110) CometHashAggregate +Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] Keys: [] -Functions [3]: [sum(sales#140), sum(returns#141), sum(profit#142)] -Aggregate Attributes [3]: [sum(sales#140)#155, sum(returns#141)#156, sum(profit#142)#157] -Results [5]: [null AS channel#158, null AS id#159, sum(sales#140)#155 AS sales#160, sum(returns#141)#156 AS returns#161, sum(profit#142)#157 AS profit#162] +Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] -(119) Union +(111) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Child 1 Input [5]: [channel#76, id#118, sales#119, returns#120, profit#121] +Child 2 Input [5]: [channel#122, id#123, sales#124, returns#125, profit#126] -(120) HashAggregate [codegen id : 18] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(112) CometHashAggregate +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -(121) CometColumnarExchange -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: hashpartitioning(channel#31, id#32, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +(113) CometExchange +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Arguments: hashpartitioning(channel#76, id#77, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(122) CometHashAggregate -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(114) CometHashAggregate +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] Functions: [] -(123) CometTakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#115,returns#116,profit#117]), [channel#31, id#32, sales#115, returns#116, profit#117], 100, 0, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#115, returns#116, profit#117] +(115) CometTakeOrderedAndProject +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#115,returns#116,profit#117]), [channel#76, id#77, sales#115, returns#116, profit#117], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#115, returns#116, profit#117] -(124) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(116) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) +BroadcastExchange (121) ++- * CometColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(117) 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,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(126) CometFilter +(118) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(127) CometProject +(119) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(128) CometColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(129) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt index f3a5ab15b6..d27ab827e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a.native_iceberg_compat/simplified.txt @@ -1,156 +1,128 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - 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] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [s_store_id] #3 + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + 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 [s_store_sk,s_store_id] #8 - 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] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - WholeStageCodegen (11) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #19 - WholeStageCodegen (10) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #20 - WholeStageCodegen (16) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + 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 [s_store_sk,s_store_id] #8 + 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] #9 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #10 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [cp_catalog_page_id] #11 + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #12 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [web_site_id] #15 + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #16 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #19 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange #20 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt index 7026380e53..f9901f13d7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/explain.txt @@ -1,128 +1,120 @@ == Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometHashAggregate (122) - +- CometColumnarExchange (121) - +- * HashAggregate (120) - +- Union (119) - :- * HashAggregate (104) - : +- * CometColumnarToRow (103) - : +- CometColumnarExchange (102) - : +- * HashAggregate (101) - : +- Union (100) - : :- * HashAggregate (39) - : : +- * CometColumnarToRow (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) - : : +- CometExchange (67) - : : +- CometHashAggregate (66) - : : +- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (62) - : : : +- CometBroadcastHashJoin (61) - : : : :- CometProject (59) - : : : : +- CometBroadcastHashJoin (58) - : : : : :- CometProject (53) - : : : : : +- CometBroadcastHashJoin (52) - : : : : : :- CometProject (50) - : : : : : : +- CometSortMergeJoin (49) - : : : : : : :- CometSort (43) - : : : : : : : +- CometExchange (42) - : : : : : : : +- CometFilter (41) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- CometSort (48) - : : : : : : +- CometExchange (47) - : : : : : : +- CometProject (46) - : : : : : : +- CometFilter (45) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : : +- ReusedExchange (51) - : : : : +- CometBroadcastExchange (57) - : : : : +- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : : +- ReusedExchange (60) - : : +- ReusedExchange (63) - : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) - : +- CometExchange (97) - : +- CometHashAggregate (96) - : +- CometProject (95) - : +- CometBroadcastHashJoin (94) - : :- CometProject (92) - : : +- CometBroadcastHashJoin (91) - : : :- CometProject (89) - : : : +- CometBroadcastHashJoin (88) - : : : :- CometProject (83) - : : : : +- CometBroadcastHashJoin (82) - : : : : :- CometProject (80) - : : : : : +- CometSortMergeJoin (79) - : : : : : :- CometSort (73) - : : : : : : +- CometExchange (72) - : : : : : : +- CometFilter (71) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : : +- CometSort (78) - : : : : : +- CometExchange (77) - : : : : : +- CometProject (76) - : : : : : +- CometFilter (75) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : : +- ReusedExchange (81) - : : : +- CometBroadcastExchange (87) - : : : +- CometProject (86) - : : : +- CometFilter (85) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (111) - : +- * CometColumnarToRow (110) - : +- CometColumnarExchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * CometColumnarToRow (106) - : +- ReusedExchange (105) - +- * HashAggregate (118) - +- * CometColumnarToRow (117) - +- CometColumnarExchange (116) - +- * HashAggregate (115) - +- * HashAggregate (114) - +- * CometColumnarToRow (113) - +- ReusedExchange (112) +* CometColumnarToRow (116) ++- CometTakeOrderedAndProject (115) + +- CometHashAggregate (114) + +- CometExchange (113) + +- CometHashAggregate (112) + +- CometUnion (111) + :- CometHashAggregate (100) + : +- CometExchange (99) + : +- CometHashAggregate (98) + : +- CometUnion (97) + : :- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (29) + : : : +- CometBroadcastHashJoin (28) + : : : :- CometProject (23) + : : : : +- CometBroadcastHashJoin (22) + : : : : :- CometProject (17) + : : : : : +- CometBroadcastHashJoin (16) + : : : : : :- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : : +- CometBroadcastExchange (15) + : : : : : +- CometProject (14) + : : : : : +- CometFilter (13) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) + : : : : +- CometBroadcastExchange (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) + : : : +- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + : : +- CometBroadcastExchange (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + : :- CometHashAggregate (67) + : : +- CometExchange (66) + : : +- CometHashAggregate (65) + : : +- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometProject (61) + : : : +- CometBroadcastHashJoin (60) + : : : :- CometProject (58) + : : : : +- CometBroadcastHashJoin (57) + : : : : :- CometProject (52) + : : : : : +- CometBroadcastHashJoin (51) + : : : : : :- CometProject (49) + : : : : : : +- CometSortMergeJoin (48) + : : : : : : :- CometSort (42) + : : : : : : : +- CometExchange (41) + : : : : : : : +- CometFilter (40) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- CometSort (47) + : : : : : : +- CometExchange (46) + : : : : : : +- CometProject (45) + : : : : : : +- CometFilter (44) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : : : : +- ReusedExchange (50) + : : : : +- CometBroadcastExchange (56) + : : : : +- CometProject (55) + : : : : +- CometFilter (54) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) + : : : +- ReusedExchange (59) + : : +- ReusedExchange (62) + : +- CometHashAggregate (96) + : +- CometExchange (95) + : +- CometHashAggregate (94) + : +- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometProject (81) + : : : : +- CometBroadcastHashJoin (80) + : : : : :- CometProject (78) + : : : : : +- CometSortMergeJoin (77) + : : : : : :- CometSort (71) + : : : : : : +- CometExchange (70) + : : : : : : +- CometFilter (69) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) + : : : : : +- CometSort (76) + : : : : : +- CometExchange (75) + : : : : : +- CometProject (74) + : : : : : +- CometFilter (73) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) + : : : : +- ReusedExchange (79) + : : : +- CometBroadcastExchange (85) + : : : +- CometProject (84) + : : : +- CometFilter (83) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) + : : +- ReusedExchange (88) + : +- ReusedExchange (91) + :- CometHashAggregate (105) + : +- CometExchange (104) + : +- CometHashAggregate (103) + : +- CometHashAggregate (102) + : +- ReusedExchange (101) + +- CometHashAggregate (110) + +- CometExchange (109) + +- CometHashAggregate (108) + +- CometHashAggregate (107) + +- ReusedExchange (106) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -298,436 +290,394 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(38) CometHashAggregate Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Keys [1]: [s_store_id#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [store channel AS channel#31, concat(store, s_store_id#18) AS id#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#33, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#34, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) -(42) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(41) CometExchange +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 ASC NULLS FIRST] +(42) CometSort +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(44) CometFilter +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) -(46) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(45) CometProject +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(46) CometExchange +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 ASC NULLS FIRST] +(47) CometSort +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], LeftOuter +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(49) CometProject +Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(52) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#51, 16, true, false, true) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(55) CometProject +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(56) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(57) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(58) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(60) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [i_item_sk#45] +Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(61) CometProject +Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] +Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(63) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [p_promo_sk#46] +Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(64) CometProject +Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] +Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(65) CometHashAggregate +Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(66) CometExchange +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [catalog channel AS channel#63, concat(catalog_page, cp_catalog_page_id#52) AS id#64, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#65, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#66, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#67] +(67) CometHashAggregate +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(69) CometFilter +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(70) CometExchange +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 ASC NULLS FIRST] +(71) CometSort +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(73) CometFilter +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(74) CometProject +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(75) CometExchange +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(76) CometSort +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], LeftOuter +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(78) CometProject +Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +Right output [1]: [d_date_sk#65] +Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(81) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#66, web_site_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#83, 16, true, false, true) AS web_site_id#84] +(84) CometProject +Input [2]: [web_site_sk#66, web_site_id#67] +Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +Right output [2]: [web_site_sk#66, web_site_id#68] +Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(87) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] +Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [i_item_sk#69] +Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(90) CometProject +Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] +Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(92) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [p_promo_sk#70] +Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(93) CometProject +Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] +Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(94) CometHashAggregate +Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Keys [1]: [web_site_id#68] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(95) CometExchange +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] (96) CometHashAggregate -Input [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [web channel AS channel#95, concat(web_site, web_site_id#84) AS id#96, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#97, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#98, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#99] - -(100) Union - -(101) HashAggregate [codegen id : 4] -Input [5]: [channel#31, id#32, sales#33, returns#34, profit#35] -Keys [2]: [channel#31, id#32] -Functions [3]: [partial_sum(sales#33), partial_sum(returns#34), partial_sum(profit#35)] -Aggregate Attributes [6]: [sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Results [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(102) CometColumnarExchange -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(103) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(104) HashAggregate [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [5]: [channel#31, id#32, cast(sum(sales#33)#112 as decimal(37,2)) AS sales#115, cast(sum(returns#34)#113 as decimal(38,2)) AS returns#116, cast(sum(profit#35)#114 as decimal(38,2)) AS profit#117] - -(105) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(106) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(107) HashAggregate [codegen id : 10] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [4]: [channel#31, sum(sales#33)#112 AS sales#118, sum(returns#34)#113 AS returns#119, sum(profit#35)#114 AS profit#120] - -(108) HashAggregate [codegen id : 10] -Input [4]: [channel#31, sales#118, returns#119, profit#120] -Keys [1]: [channel#31] -Functions [3]: [partial_sum(sales#118), partial_sum(returns#119), partial_sum(profit#120)] -Aggregate Attributes [6]: [sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Results [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(109) CometColumnarExchange -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(110) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(111) HashAggregate [codegen id : 11] -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [1]: [channel#31] -Functions [3]: [sum(sales#118), sum(returns#119), sum(profit#120)] -Aggregate Attributes [3]: [sum(sales#118)#133, sum(returns#119)#134, sum(profit#120)#135] -Results [5]: [channel#31, null AS id#136, sum(sales#118)#133 AS sales#137, sum(returns#119)#134 AS returns#138, sum(profit#120)#135 AS profit#139] - -(112) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(113) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(114) HashAggregate [codegen id : 16] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [3]: [sum(sales#33)#112 AS sales#140, sum(returns#34)#113 AS returns#141, sum(profit#35)#114 AS profit#142] - -(115) HashAggregate [codegen id : 16] -Input [3]: [sales#140, returns#141, profit#142] +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Keys [1]: [web_site_id#68] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(97) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] +Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] + +(98) CometHashAggregate +Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Keys [2]: [channel#76, id#77] +Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] + +(99) CometExchange +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(100) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(101) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(102) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(103) CometHashAggregate +Input [4]: [channel#76, sales#97, returns#98, profit#99] +Keys [1]: [channel#76] +Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] + +(104) CometExchange +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(105) CometHashAggregate +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Keys [1]: [channel#76] +Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] + +(106) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(107) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(108) CometHashAggregate +Input [3]: [sales#106, returns#107, profit#108] Keys: [] -Functions [3]: [partial_sum(sales#140), partial_sum(returns#141), partial_sum(profit#142)] -Aggregate Attributes [6]: [sum#143, isEmpty#144, sum#145, isEmpty#146, sum#147, isEmpty#148] -Results [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] - -(116) CometColumnarExchange -Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Functions [3]: [partial_sum(sales#106), partial_sum(returns#107), partial_sum(profit#108)] -(117) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] +(109) CometExchange +Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(118) HashAggregate [codegen id : 17] -Input [6]: [sum#149, isEmpty#150, sum#151, isEmpty#152, sum#153, isEmpty#154] +(110) CometHashAggregate +Input [6]: [sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] Keys: [] -Functions [3]: [sum(sales#140), sum(returns#141), sum(profit#142)] -Aggregate Attributes [3]: [sum(sales#140)#155, sum(returns#141)#156, sum(profit#142)#157] -Results [5]: [null AS channel#158, null AS id#159, sum(sales#140)#155 AS sales#160, sum(returns#141)#156 AS returns#161, sum(profit#142)#157 AS profit#162] +Functions [3]: [sum(sales#106), sum(returns#107), sum(profit#108)] -(119) Union +(111) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Child 1 Input [5]: [channel#76, id#118, sales#119, returns#120, profit#121] +Child 2 Input [5]: [channel#122, id#123, sales#124, returns#125, profit#126] -(120) HashAggregate [codegen id : 18] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(112) CometHashAggregate +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -(121) CometColumnarExchange -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: hashpartitioning(channel#31, id#32, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +(113) CometExchange +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Arguments: hashpartitioning(channel#76, id#77, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(122) CometHashAggregate -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(114) CometHashAggregate +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Keys [5]: [channel#76, id#77, sales#115, returns#116, profit#117] Functions: [] -(123) CometTakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#115,returns#116,profit#117]), [channel#31, id#32, sales#115, returns#116, profit#117], 100, 0, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#115, returns#116, profit#117] +(115) CometTakeOrderedAndProject +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#115,returns#116,profit#117]), [channel#76, id#77, sales#115, returns#116, profit#117], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#115, returns#116, profit#117] -(124) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(116) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#76, id#77, sales#115, returns#116, profit#117] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) +BroadcastExchange (121) ++- * CometColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(117) 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,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(126) CometFilter +(118) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(127) CometProject +(119) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(128) CometColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(129) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt index f3a5ab15b6..d27ab827e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7-spark3_5/q80a/simplified.txt @@ -1,156 +1,128 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - 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] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [s_store_id] #3 + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + 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 [s_store_sk,s_store_id] #8 - 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] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - WholeStageCodegen (11) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #19 - WholeStageCodegen (10) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #20 - WholeStageCodegen (16) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + 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 [s_store_sk,s_store_id] #8 + 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] #9 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #10 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [cp_catalog_page_id] #11 + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #12 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [web_site_id] #15 + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #16 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #19 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange #20 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 From a9ca6518b849c73fd5dda3cded4fb962b21199f6 Mon Sep 17 00:00:00 2001 From: comphead Date: Mon, 27 Oct 2025 11:06:32 -0700 Subject: [PATCH 7/8] feat: support `concat` for strings --- .../q84.native_iceberg_compat/explain.txt | 20 +- .../q84.native_iceberg_compat/simplified.txt | 10 +- .../q84/explain.txt | 20 +- .../q84/simplified.txt | 10 +- .../q5.native_iceberg_compat/explain.txt | 513 ++++++----- .../q5.native_iceberg_compat/simplified.txt | 183 ++-- .../approved-plans-v1_4/q5/explain.txt | 513 ++++++----- .../approved-plans-v1_4/q5/simplified.txt | 183 ++-- .../q80.native_iceberg_compat/explain.txt | 669 +++++++------- .../q80.native_iceberg_compat/simplified.txt | 241 +++-- .../approved-plans-v1_4/q80/explain.txt | 669 +++++++------- .../approved-plans-v1_4/q80/simplified.txt | 241 +++-- .../q84.native_iceberg_compat/explain.txt | 20 +- .../q84.native_iceberg_compat/simplified.txt | 10 +- .../approved-plans-v1_4/q84/explain.txt | 20 +- .../approved-plans-v1_4/q84/simplified.txt | 10 +- .../q5a.native_iceberg_compat/explain.txt | 664 +++++++------- .../q5a.native_iceberg_compat/simplified.txt | 218 ++--- .../approved-plans-v2_7/q5a/explain.txt | 664 +++++++------- .../approved-plans-v2_7/q5a/simplified.txt | 218 ++--- .../q80a.native_iceberg_compat/explain.txt | 820 ++++++++---------- .../q80a.native_iceberg_compat/simplified.txt | 270 +++--- .../approved-plans-v2_7/q80a/explain.txt | 820 ++++++++---------- .../approved-plans-v2_7/q80a/simplified.txt | 270 +++--- 24 files changed, 3422 insertions(+), 3854 deletions(-) diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/explain.txt index c340659028..ec24951771 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * Project (32) - +- * CometColumnarToRow (31) +* CometColumnarToRow (33) ++- CometTakeOrderedAndProject (32) + +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometBroadcastExchange (26) : +- CometProject (25) @@ -177,14 +177,14 @@ Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] Right output [1]: [sr_cdemo_sk#18] Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft -(31) CometColumnarToRow [codegen id : 1] +(31) CometProject Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] +Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -(32) Project [codegen id : 1] -Output [3]: [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] - -(33) TakeOrderedAndProject +(32) CometTakeOrderedAndProject Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: 100, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] + +(33) CometColumnarToRow [codegen id : 1] +Input [2]: [customer_id#20, customername#21] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/simplified.txt index cc7cf901e5..fd8d1864ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [c_customer_id,customer_id,customername] - WholeStageCodegen (1) - Project [c_customer_id,c_last_name,c_first_name] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customername,c_customer_id] + CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt index c340659028..ec24951771 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * Project (32) - +- * CometColumnarToRow (31) +* CometColumnarToRow (33) ++- CometTakeOrderedAndProject (32) + +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometBroadcastExchange (26) : +- CometProject (25) @@ -177,14 +177,14 @@ Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] Right output [1]: [sr_cdemo_sk#18] Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft -(31) CometColumnarToRow [codegen id : 1] +(31) CometProject Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] +Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -(32) Project [codegen id : 1] -Output [3]: [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] - -(33) TakeOrderedAndProject +(32) CometTakeOrderedAndProject Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: 100, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] + +(33) CometColumnarToRow [codegen id : 1] +Input [2]: [customer_id#20, customername#21] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt index cc7cf901e5..fd8d1864ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4-spark4_0/q84/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [c_customer_id,customer_id,customername] - WholeStageCodegen (1) - Project [c_customer_id,c_last_name,c_first_name] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customername,c_customer_id] + CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/explain.txt index f2dce3f729..b4a1dedd61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/explain.txt @@ -1,78 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Expand (69) - +- Union (68) - :- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- * HashAggregate (43) - : +- * CometColumnarToRow (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (33) - : : +- CometBroadcastHashJoin (32) - : : :- CometUnion (30) - : : : :- CometProject (26) - : : : : +- CometFilter (25) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : +- ReusedExchange (31) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - +- * HashAggregate (67) - +- * CometColumnarToRow (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometProject (63) - +- CometBroadcastHashJoin (62) - :- CometProject (57) - : +- CometBroadcastHashJoin (56) - : :- CometUnion (54) - : : :- CometProject (46) - : : : +- CometFilter (45) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : +- CometProject (53) - : : +- CometBroadcastHashJoin (52) - : : :- CometBroadcastExchange (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : +- CometProject (51) - : : +- CometFilter (50) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : +- ReusedExchange (55) - +- CometBroadcastExchange (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) +* CometColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometHashAggregate (69) + +- CometExchange (68) + +- CometHashAggregate (67) + +- CometExpand (66) + +- CometUnion (65) + :- CometHashAggregate (22) + : +- CometExchange (21) + : +- CometHashAggregate (20) + : +- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometUnion (7) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : +- CometBroadcastExchange (11) + : : +- CometProject (10) + : : +- CometFilter (9) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) + : +- CometBroadcastExchange (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + :- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometUnion (29) + : : : :- CometProject (25) + : : : : +- CometFilter (24) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : +- ReusedExchange (30) + : +- CometBroadcastExchange (36) + : +- CometProject (35) + : +- CometFilter (34) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometUnion (52) + : : :- CometProject (44) + : : : +- CometFilter (43) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometBroadcastExchange (46) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : +- ReusedExchange (53) + +- CometBroadcastExchange (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -176,296 +173,280 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Keys [1]: [s_store_id#26] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#35, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#36, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#37, store channel AS channel#38, concat(store, s_store_id#26) AS id#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : isnotnull(cs_catalog_page_sk#31) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(25) CometProject +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Condition : isnotnull(cr_catalog_page_sk#42) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(28) CometProject +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(29) CometUnion +Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [1]: [d_date_sk#52] +Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight -(33) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(32) CometProject +Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] +Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#63, 16, true, false, true) AS cp_catalog_page_id#64] +(35) CometProject +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(36) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(37) CometBroadcastHashJoin +Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(38) CometProject +Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(39) CometHashAggregate +Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometExchange +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(41) CometHashAggregate +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#73, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#74, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#75, catalog channel AS channel#76, concat(catalog_page, cp_catalog_page_id#64) AS id#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(43) CometFilter +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_web_site_sk#60) -(46) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(44) CometProject +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(48) CometFilter +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(49) CometProject +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#103] +(51) CometProject +Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [1]: [d_date_sk#86] +Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(55) CometProject +Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] +Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#87, web_site_id#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(57) CometFilter +Input [2]: [web_site_sk#87, web_site_id#88] +Condition : isnotnull(web_site_sk#87) + +(58) CometProject +Input [2]: [web_site_sk#87, web_site_id#88] +Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] + +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] -(60) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#106, 16, true, false, true) AS web_site_id#107] +(60) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [2]: [web_site_sk#87, web_site_id#89] +Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight -(61) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(61) CometProject +Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] +Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -(62) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], Inner, BuildRight +(62) CometHashAggregate +Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] +Keys [1]: [web_site_id#89] +Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] -(63) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(63) CometExchange +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (64) CometHashAggregate -Input [5]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#116, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#117, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#118, web channel AS channel#119, concat(web_site, web_site_id#107) AS id#120] - -(68) Union - -(69) Expand [codegen id : 4] -Input [5]: [sales#35, returns#36, profit#37, channel#38, id#39] -Arguments: [[sales#35, returns#36, profit#37, channel#38, id#39, 0], [sales#35, returns#36, profit#37, channel#38, null, 1], [sales#35, returns#36, profit#37, null, null, 3]], [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] - -(70) HashAggregate [codegen id : 4] -Input [6]: [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [partial_sum(sales#35), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Results [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(71) CometColumnarExchange -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Arguments: hashpartitioning(channel#121, id#122, spark_grouping_id#123, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(72) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(73) HashAggregate [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [sum(sales#35), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#35)#136, sum(returns#36)#137, sum(profit#37)#138] -Results [5]: [channel#121, id#122, sum(sales#35)#136 AS sales#139, sum(returns#36)#137 AS returns#140, sum(profit#37)#138 AS profit#141] - -(74) TakeOrderedAndProject -Input [5]: [channel#121, id#122, sales#139, returns#140, profit#141] -Arguments: 100, [channel#121 ASC NULLS FIRST, id#122 ASC NULLS FIRST], [channel#121, id#122, sales#139, returns#140, profit#141] +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Keys [1]: [web_site_id#89] +Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] + +(65) CometUnion +Child 0 Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] +Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] +Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] + +(66) CometExpand +Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] +Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] + +(67) CometHashAggregate +Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] + +(68) CometExchange +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(69) CometHashAggregate +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] + +(70) CometTakeOrderedAndProject +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] + +(71) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter +(73) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(77) CometProject +(74) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(79) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/simplified.txt index e86e0869f5..6fff89a25c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.native_iceberg_compat/simplified.txt @@ -1,97 +1,86 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [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_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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #4 - 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 [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] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #8 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [web_site_sk,web_site_id] #10 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] + CometExchange [s_store_id] #2 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [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_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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #4 + 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 [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] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #6 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] + CometExchange [web_site_id] #8 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt index f2dce3f729..b4a1dedd61 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt @@ -1,78 +1,75 @@ == Physical Plan == -TakeOrderedAndProject (74) -+- * HashAggregate (73) - +- * CometColumnarToRow (72) - +- CometColumnarExchange (71) - +- * HashAggregate (70) - +- * Expand (69) - +- Union (68) - :- * HashAggregate (23) - : +- * CometColumnarToRow (22) - : +- CometExchange (21) - : +- CometHashAggregate (20) - : +- CometProject (19) - : +- CometBroadcastHashJoin (18) - : :- CometProject (13) - : : +- CometBroadcastHashJoin (12) - : : :- CometUnion (7) - : : : :- CometProject (3) - : : : : +- CometFilter (2) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : +- CometProject (6) - : : : +- CometFilter (5) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : +- CometBroadcastExchange (11) - : : +- CometProject (10) - : : +- CometFilter (9) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : +- CometBroadcastExchange (17) - : +- CometProject (16) - : +- CometFilter (15) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - :- * HashAggregate (43) - : +- * CometColumnarToRow (42) - : +- CometExchange (41) - : +- CometHashAggregate (40) - : +- CometProject (39) - : +- CometBroadcastHashJoin (38) - : :- CometProject (33) - : : +- CometBroadcastHashJoin (32) - : : :- CometUnion (30) - : : : :- CometProject (26) - : : : : +- CometFilter (25) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : +- CometProject (29) - : : : +- CometFilter (28) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : +- ReusedExchange (31) - : +- CometBroadcastExchange (37) - : +- CometProject (36) - : +- CometFilter (35) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - +- * HashAggregate (67) - +- * CometColumnarToRow (66) - +- CometExchange (65) - +- CometHashAggregate (64) - +- CometProject (63) - +- CometBroadcastHashJoin (62) - :- CometProject (57) - : +- CometBroadcastHashJoin (56) - : :- CometUnion (54) - : : :- CometProject (46) - : : : +- CometFilter (45) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : +- CometProject (53) - : : +- CometBroadcastHashJoin (52) - : : :- CometBroadcastExchange (48) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : +- CometProject (51) - : : +- CometFilter (50) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : +- ReusedExchange (55) - +- CometBroadcastExchange (61) - +- CometProject (60) - +- CometFilter (59) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) +* CometColumnarToRow (71) ++- CometTakeOrderedAndProject (70) + +- CometHashAggregate (69) + +- CometExchange (68) + +- CometHashAggregate (67) + +- CometExpand (66) + +- CometUnion (65) + :- CometHashAggregate (22) + : +- CometExchange (21) + : +- CometHashAggregate (20) + : +- CometProject (19) + : +- CometBroadcastHashJoin (18) + : :- CometProject (13) + : : +- CometBroadcastHashJoin (12) + : : :- CometUnion (7) + : : : :- CometProject (3) + : : : : +- CometFilter (2) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : +- CometProject (6) + : : : +- CometFilter (5) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : +- CometBroadcastExchange (11) + : : +- CometProject (10) + : : +- CometFilter (9) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) + : +- CometBroadcastExchange (17) + : +- CometProject (16) + : +- CometFilter (15) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + :- CometHashAggregate (41) + : +- CometExchange (40) + : +- CometHashAggregate (39) + : +- CometProject (38) + : +- CometBroadcastHashJoin (37) + : :- CometProject (32) + : : +- CometBroadcastHashJoin (31) + : : :- CometUnion (29) + : : : :- CometProject (25) + : : : : +- CometFilter (24) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : +- CometProject (28) + : : : +- CometFilter (27) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : +- ReusedExchange (30) + : +- CometBroadcastExchange (36) + : +- CometProject (35) + : +- CometFilter (34) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) + +- CometHashAggregate (64) + +- CometExchange (63) + +- CometHashAggregate (62) + +- CometProject (61) + +- CometBroadcastHashJoin (60) + :- CometProject (55) + : +- CometBroadcastHashJoin (54) + : :- CometUnion (52) + : : :- CometProject (44) + : : : +- CometFilter (43) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : +- CometProject (51) + : : +- CometBroadcastHashJoin (50) + : : :- CometBroadcastExchange (46) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) + : : +- CometProject (49) + : : +- CometFilter (48) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : +- ReusedExchange (53) + +- CometBroadcastExchange (59) + +- CometProject (58) + +- CometFilter (57) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -176,296 +173,280 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Keys [1]: [s_store_id#26] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#35, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#36, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#37, store channel AS channel#38, concat(store, s_store_id#26) AS id#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : isnotnull(cs_catalog_page_sk#31) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(25) CometProject +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Condition : isnotnull(cr_catalog_page_sk#42) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(28) CometProject +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(29) CometUnion +Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [1]: [d_date_sk#52] +Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight -(33) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(32) CometProject +Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] +Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#63, 16, true, false, true) AS cp_catalog_page_id#64] +(35) CometProject +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(36) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(37) CometBroadcastHashJoin +Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(38) CometProject +Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(39) CometHashAggregate +Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometExchange +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(41) CometHashAggregate +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#73, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#74, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#75, catalog channel AS channel#76, concat(catalog_page, cp_catalog_page_id#64) AS id#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(43) CometFilter +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_web_site_sk#60) -(46) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(44) CometProject +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(48) CometFilter +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(49) CometProject +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#103] +(51) CometProject +Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [1]: [d_date_sk#86] +Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(55) CometProject +Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] +Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#87, web_site_id#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(57) CometFilter +Input [2]: [web_site_sk#87, web_site_id#88] +Condition : isnotnull(web_site_sk#87) + +(58) CometProject +Input [2]: [web_site_sk#87, web_site_id#88] +Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] + +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] -(60) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#106, 16, true, false, true) AS web_site_id#107] +(60) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [2]: [web_site_sk#87, web_site_id#89] +Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight -(61) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(61) CometProject +Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] +Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -(62) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], Inner, BuildRight +(62) CometHashAggregate +Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] +Keys [1]: [web_site_id#89] +Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] -(63) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(63) CometExchange +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (64) CometHashAggregate -Input [5]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#116, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#117, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#118, web channel AS channel#119, concat(web_site, web_site_id#107) AS id#120] - -(68) Union - -(69) Expand [codegen id : 4] -Input [5]: [sales#35, returns#36, profit#37, channel#38, id#39] -Arguments: [[sales#35, returns#36, profit#37, channel#38, id#39, 0], [sales#35, returns#36, profit#37, channel#38, null, 1], [sales#35, returns#36, profit#37, null, null, 3]], [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] - -(70) HashAggregate [codegen id : 4] -Input [6]: [sales#35, returns#36, profit#37, channel#121, id#122, spark_grouping_id#123] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [partial_sum(sales#35), partial_sum(returns#36), partial_sum(profit#37)] -Aggregate Attributes [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] -Results [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(71) CometColumnarExchange -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Arguments: hashpartitioning(channel#121, id#122, spark_grouping_id#123, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(72) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] - -(73) HashAggregate [codegen id : 5] -Input [9]: [channel#121, id#122, spark_grouping_id#123, sum#130, isEmpty#131, sum#132, isEmpty#133, sum#134, isEmpty#135] -Keys [3]: [channel#121, id#122, spark_grouping_id#123] -Functions [3]: [sum(sales#35), sum(returns#36), sum(profit#37)] -Aggregate Attributes [3]: [sum(sales#35)#136, sum(returns#36)#137, sum(profit#37)#138] -Results [5]: [channel#121, id#122, sum(sales#35)#136 AS sales#139, sum(returns#36)#137 AS returns#140, sum(profit#37)#138 AS profit#141] - -(74) TakeOrderedAndProject -Input [5]: [channel#121, id#122, sales#139, returns#140, profit#141] -Arguments: 100, [channel#121 ASC NULLS FIRST, id#122 ASC NULLS FIRST], [channel#121, id#122, sales#139, returns#140, profit#141] +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Keys [1]: [web_site_id#89] +Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] + +(65) CometUnion +Child 0 Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] +Child 1 Input [5]: [sales#99, returns#100, profit#101, channel#102, id#103] +Child 2 Input [5]: [sales#104, returns#105, profit#106, channel#107, id#108] + +(66) CometExpand +Input [5]: [sales#94, returns#95, profit#96, channel#97, id#98] +Arguments: [[sales#94, returns#95, profit#96, channel#97, id#98, 0], [sales#94, returns#95, profit#96, channel#97, null, 1], [sales#94, returns#95, profit#96, null, null, 3]], [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] + +(67) CometHashAggregate +Input [6]: [sales#94, returns#95, profit#96, channel#109, id#110, spark_grouping_id#111] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [partial_sum(sales#94), partial_sum(returns#95), partial_sum(profit#96)] + +(68) CometExchange +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Arguments: hashpartitioning(channel#109, id#110, spark_grouping_id#111, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(69) CometHashAggregate +Input [9]: [channel#109, id#110, spark_grouping_id#111, sum#112, isEmpty#113, sum#114, isEmpty#115, sum#116, isEmpty#117] +Keys [3]: [channel#109, id#110, spark_grouping_id#111] +Functions [3]: [sum(sales#94), sum(returns#95), sum(profit#96)] + +(70) CometTakeOrderedAndProject +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#109 ASC NULLS FIRST,id#110 ASC NULLS FIRST], output=[channel#109,id#110,sales#118,returns#119,profit#120]), [channel#109, id#110, sales#118, returns#119, profit#120], 100, 0, [channel#109 ASC NULLS FIRST, id#110 ASC NULLS FIRST], [channel#109, id#110, sales#118, returns#119, profit#120] + +(71) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#109, id#110, sales#118, returns#119, profit#120] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (79) -+- * CometColumnarToRow (78) - +- CometProject (77) - +- CometFilter (76) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (75) +BroadcastExchange (76) ++- * CometColumnarToRow (75) + +- CometProject (74) + +- CometFilter (73) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (72) -(75) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,2000-08-23), LessThanOrEqual(d_date,2000-09-06), IsNotNull(d_date_sk)] ReadSchema: struct -(76) CometFilter +(73) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 2000-08-23)) AND (d_date#23 <= 2000-09-06)) AND isnotnull(d_date_sk#22)) -(77) CometProject +(74) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(78) CometColumnarToRow [codegen id : 1] +(75) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(79) BroadcastExchange +(76) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt index e86e0869f5..6fff89a25c 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/simplified.txt @@ -1,97 +1,86 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [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_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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometBroadcastExchange [d_date_sk] #4 - 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 [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] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #6 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),sales,returns,profit,channel,id,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #8 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #4 - CometBroadcastExchange [web_site_sk,web_site_id] #10 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),s_store_id] + CometExchange [s_store_id] #2 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [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_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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometBroadcastExchange [d_date_sk] #4 + 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 [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] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #6 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #7 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + CometHashAggregate [sum,sum,sum,sum] [sales,returns,profit,channel,id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),web_site_id] + CometExchange [web_site_id] #8 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #9 + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #4 + CometBroadcastExchange [web_site_sk,web_site_id] #10 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/explain.txt index bbc01dd8aa..f0a163bf69 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/explain.txt @@ -1,110 +1,107 @@ == Physical Plan == -TakeOrderedAndProject (106) -+- * HashAggregate (105) - +- * CometColumnarToRow (104) - +- CometColumnarExchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * HashAggregate (39) - : +- * CometColumnarToRow (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometProject (17) - : : : : +- CometBroadcastHashJoin (16) - : : : : :- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : +- CometBroadcastExchange (15) - : : : : +- CometProject (14) - : : : : +- CometFilter (13) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : +- CometBroadcastExchange (21) - : : : +- CometProject (20) - : : : +- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - :- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (53) - : : : : +- CometBroadcastHashJoin (52) - : : : : :- CometProject (50) - : : : : : +- CometSortMergeJoin (49) - : : : : : :- CometSort (43) - : : : : : : +- CometExchange (42) - : : : : : : +- CometFilter (41) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : +- CometSort (48) - : : : : : +- CometExchange (47) - : : : : : +- CometProject (46) - : : : : : +- CometFilter (45) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : +- ReusedExchange (51) - : : : +- CometBroadcastExchange (57) - : : : +- CometProject (56) - : : : +- CometFilter (55) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- * HashAggregate (99) - +- * CometColumnarToRow (98) - +- CometExchange (97) - +- CometHashAggregate (96) - +- CometProject (95) - +- CometBroadcastHashJoin (94) - :- CometProject (92) - : +- CometBroadcastHashJoin (91) - : :- CometProject (89) - : : +- CometBroadcastHashJoin (88) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (80) - : : : : +- CometSortMergeJoin (79) - : : : : :- CometSort (73) - : : : : : +- CometExchange (72) - : : : : : +- CometFilter (71) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : +- CometSort (78) - : : : : +- CometExchange (77) - : : : : +- CometProject (76) - : : : : +- CometFilter (75) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : +- ReusedExchange (81) - : : +- CometBroadcastExchange (87) - : : +- CometProject (86) - : : +- CometFilter (85) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : +- ReusedExchange (90) - +- ReusedExchange (93) +* CometColumnarToRow (103) ++- CometTakeOrderedAndProject (102) + +- CometHashAggregate (101) + +- CometExchange (100) + +- CometHashAggregate (99) + +- CometExpand (98) + +- CometUnion (97) + :- CometHashAggregate (38) + : +- CometExchange (37) + : +- CometHashAggregate (36) + : +- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometProject (17) + : : : : +- CometBroadcastHashJoin (16) + : : : : :- CometProject (11) + : : : : : +- CometSortMergeJoin (10) + : : : : : :- CometSort (4) + : : : : : : +- CometExchange (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : +- CometSort (9) + : : : : : +- CometExchange (8) + : : : : : +- CometProject (7) + : : : : : +- CometFilter (6) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : +- CometBroadcastExchange (15) + : : : : +- CometProject (14) + : : : : +- CometFilter (13) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) + : : : +- CometBroadcastExchange (21) + : : : +- CometProject (20) + : : : +- CometFilter (19) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) + : : +- CometBroadcastExchange (27) + : : +- CometProject (26) + : : +- CometFilter (25) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + : +- CometBroadcastExchange (33) + : +- CometProject (32) + : +- CometFilter (31) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + :- CometHashAggregate (67) + : +- CometExchange (66) + : +- CometHashAggregate (65) + : +- CometProject (64) + : +- CometBroadcastHashJoin (63) + : :- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (58) + : : : +- CometBroadcastHashJoin (57) + : : : :- CometProject (52) + : : : : +- CometBroadcastHashJoin (51) + : : : : :- CometProject (49) + : : : : : +- CometSortMergeJoin (48) + : : : : : :- CometSort (42) + : : : : : : +- CometExchange (41) + : : : : : : +- CometFilter (40) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) + : : : : : +- CometSort (47) + : : : : : +- CometExchange (46) + : : : : : +- CometProject (45) + : : : : : +- CometFilter (44) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : : : +- ReusedExchange (50) + : : : +- CometBroadcastExchange (56) + : : : +- CometProject (55) + : : : +- CometFilter (54) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) + : : +- ReusedExchange (59) + : +- ReusedExchange (62) + +- CometHashAggregate (96) + +- CometExchange (95) + +- CometHashAggregate (94) + +- CometProject (93) + +- CometBroadcastHashJoin (92) + :- CometProject (90) + : +- CometBroadcastHashJoin (89) + : :- CometProject (87) + : : +- CometBroadcastHashJoin (86) + : : :- CometProject (81) + : : : +- CometBroadcastHashJoin (80) + : : : :- CometProject (78) + : : : : +- CometSortMergeJoin (77) + : : : : :- CometSort (71) + : : : : : +- CometExchange (70) + : : : : : +- CometFilter (69) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) + : : : : +- CometSort (76) + : : : : +- CometExchange (75) + : : : : +- CometProject (74) + : : : : +- CometFilter (73) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) + : : : +- ReusedExchange (79) + : : +- CometBroadcastExchange (85) + : : +- CometProject (84) + : : +- CometFilter (83) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) + : +- ReusedExchange (88) + +- ReusedExchange (91) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -280,351 +277,335 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(38) CometHashAggregate Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Keys [1]: [s_store_id#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#33, store channel AS channel#34, concat(store, s_store_id#18) AS id#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) -(42) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(41) CometExchange +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 ASC NULLS FIRST] +(42) CometSort +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(44) CometFilter +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) -(46) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(45) CometProject +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(46) CometExchange +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 ASC NULLS FIRST] +(47) CometSort +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], LeftOuter +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(49) CometProject +Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(52) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#51, 16, true, false, true) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(55) CometProject +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(56) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(57) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(58) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(60) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [i_item_sk#45] +Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(61) CometProject +Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] +Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(63) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [p_promo_sk#46] +Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(64) CometProject +Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] +Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(65) CometHashAggregate +Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(66) CometExchange +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#63, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#64, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#65, catalog channel AS channel#66, concat(catalog_page, cp_catalog_page_id#52) AS id#67] +(67) CometHashAggregate +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(69) CometFilter +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(70) CometExchange +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 ASC NULLS FIRST] +(71) CometSort +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(73) CometFilter +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(74) CometProject +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(75) CometExchange +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(76) CometSort +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], LeftOuter +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(78) CometProject +Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +Right output [1]: [d_date_sk#65] +Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(81) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#66, web_site_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#83, 16, true, false, true) AS web_site_id#84] +(84) CometProject +Input [2]: [web_site_sk#66, web_site_id#67] +Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +Right output [2]: [web_site_sk#66, web_site_id#68] +Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(87) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] +Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [i_item_sk#69] +Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(90) CometProject +Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] +Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(92) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [p_promo_sk#70] +Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(93) CometProject +Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] +Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(94) CometHashAggregate +Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Keys [1]: [web_site_id#68] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(95) CometExchange +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] (96) CometHashAggregate -Input [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#95, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#96, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#97, web channel AS channel#98, concat(web_site, web_site_id#84) AS id#99] - -(100) Union - -(101) Expand [codegen id : 4] -Input [5]: [sales#31, returns#32, profit#33, channel#34, id#35] -Arguments: [[sales#31, returns#32, profit#33, channel#34, id#35, 0], [sales#31, returns#32, profit#33, channel#34, null, 1], [sales#31, returns#32, profit#33, null, null, 3]], [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] - -(102) HashAggregate [codegen id : 4] -Input [6]: [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [partial_sum(sales#31), partial_sum(returns#32), partial_sum(profit#33)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(103) CometColumnarExchange -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#100, id#101, spark_grouping_id#102, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(104) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(105) HashAggregate [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [sum(sales#31), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#31)#115, sum(returns#32)#116, sum(profit#33)#117] -Results [5]: [channel#100, id#101, sum(sales#31)#115 AS sales#118, sum(returns#32)#116 AS returns#119, sum(profit#33)#117 AS profit#120] - -(106) TakeOrderedAndProject -Input [5]: [channel#100, id#101, sales#118, returns#119, profit#120] -Arguments: 100, [channel#100 ASC NULLS FIRST, id#101 ASC NULLS FIRST], [channel#100, id#101, sales#118, returns#119, profit#120] +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Keys [1]: [web_site_id#68] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(97) CometUnion +Child 0 Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] +Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] +Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] + +(98) CometExpand +Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] +Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] + +(99) CometHashAggregate +Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] + +(100) CometExchange +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(101) CometHashAggregate +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] + +(102) CometTakeOrderedAndProject +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] + +(103) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometProject (109) - +- CometFilter (108) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (107) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(104) 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,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] ReadSchema: struct -(108) CometFilter +(105) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(109) CometProject +(106) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(110) CometColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(111) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/simplified.txt index 2c56b98787..198646e0fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.native_iceberg_compat/simplified.txt @@ -1,126 +1,115 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - 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] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - 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] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] + CometExchange [s_store_id] #2 + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #5 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + 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] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + 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] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #9 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #10 + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #11 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] + CometExchange [web_site_id] #14 + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #15 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #16 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt index bbc01dd8aa..f0a163bf69 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt @@ -1,110 +1,107 @@ == Physical Plan == -TakeOrderedAndProject (106) -+- * HashAggregate (105) - +- * CometColumnarToRow (104) - +- CometColumnarExchange (103) - +- * HashAggregate (102) - +- * Expand (101) - +- Union (100) - :- * HashAggregate (39) - : +- * CometColumnarToRow (38) - : +- CometExchange (37) - : +- CometHashAggregate (36) - : +- CometProject (35) - : +- CometBroadcastHashJoin (34) - : :- CometProject (29) - : : +- CometBroadcastHashJoin (28) - : : :- CometProject (23) - : : : +- CometBroadcastHashJoin (22) - : : : :- CometProject (17) - : : : : +- CometBroadcastHashJoin (16) - : : : : :- CometProject (11) - : : : : : +- CometSortMergeJoin (10) - : : : : : :- CometSort (4) - : : : : : : +- CometExchange (3) - : : : : : : +- CometFilter (2) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : +- CometSort (9) - : : : : : +- CometExchange (8) - : : : : : +- CometProject (7) - : : : : : +- CometFilter (6) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : +- CometBroadcastExchange (15) - : : : : +- CometProject (14) - : : : : +- CometFilter (13) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : +- CometBroadcastExchange (21) - : : : +- CometProject (20) - : : : +- CometFilter (19) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : +- CometBroadcastExchange (27) - : : +- CometProject (26) - : : +- CometFilter (25) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : +- CometBroadcastExchange (33) - : +- CometProject (32) - : +- CometFilter (31) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - :- * HashAggregate (69) - : +- * CometColumnarToRow (68) - : +- CometExchange (67) - : +- CometHashAggregate (66) - : +- CometProject (65) - : +- CometBroadcastHashJoin (64) - : :- CometProject (62) - : : +- CometBroadcastHashJoin (61) - : : :- CometProject (59) - : : : +- CometBroadcastHashJoin (58) - : : : :- CometProject (53) - : : : : +- CometBroadcastHashJoin (52) - : : : : :- CometProject (50) - : : : : : +- CometSortMergeJoin (49) - : : : : : :- CometSort (43) - : : : : : : +- CometExchange (42) - : : : : : : +- CometFilter (41) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : +- CometSort (48) - : : : : : +- CometExchange (47) - : : : : : +- CometProject (46) - : : : : : +- CometFilter (45) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : +- ReusedExchange (51) - : : : +- CometBroadcastExchange (57) - : : : +- CometProject (56) - : : : +- CometFilter (55) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : +- ReusedExchange (60) - : +- ReusedExchange (63) - +- * HashAggregate (99) - +- * CometColumnarToRow (98) - +- CometExchange (97) - +- CometHashAggregate (96) - +- CometProject (95) - +- CometBroadcastHashJoin (94) - :- CometProject (92) - : +- CometBroadcastHashJoin (91) - : :- CometProject (89) - : : +- CometBroadcastHashJoin (88) - : : :- CometProject (83) - : : : +- CometBroadcastHashJoin (82) - : : : :- CometProject (80) - : : : : +- CometSortMergeJoin (79) - : : : : :- CometSort (73) - : : : : : +- CometExchange (72) - : : : : : +- CometFilter (71) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : +- CometSort (78) - : : : : +- CometExchange (77) - : : : : +- CometProject (76) - : : : : +- CometFilter (75) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : +- ReusedExchange (81) - : : +- CometBroadcastExchange (87) - : : +- CometProject (86) - : : +- CometFilter (85) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : +- ReusedExchange (90) - +- ReusedExchange (93) +* CometColumnarToRow (103) ++- CometTakeOrderedAndProject (102) + +- CometHashAggregate (101) + +- CometExchange (100) + +- CometHashAggregate (99) + +- CometExpand (98) + +- CometUnion (97) + :- CometHashAggregate (38) + : +- CometExchange (37) + : +- CometHashAggregate (36) + : +- CometProject (35) + : +- CometBroadcastHashJoin (34) + : :- CometProject (29) + : : +- CometBroadcastHashJoin (28) + : : :- CometProject (23) + : : : +- CometBroadcastHashJoin (22) + : : : :- CometProject (17) + : : : : +- CometBroadcastHashJoin (16) + : : : : :- CometProject (11) + : : : : : +- CometSortMergeJoin (10) + : : : : : :- CometSort (4) + : : : : : : +- CometExchange (3) + : : : : : : +- CometFilter (2) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : +- CometSort (9) + : : : : : +- CometExchange (8) + : : : : : +- CometProject (7) + : : : : : +- CometFilter (6) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : +- CometBroadcastExchange (15) + : : : : +- CometProject (14) + : : : : +- CometFilter (13) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) + : : : +- CometBroadcastExchange (21) + : : : +- CometProject (20) + : : : +- CometFilter (19) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) + : : +- CometBroadcastExchange (27) + : : +- CometProject (26) + : : +- CometFilter (25) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + : +- CometBroadcastExchange (33) + : +- CometProject (32) + : +- CometFilter (31) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + :- CometHashAggregate (67) + : +- CometExchange (66) + : +- CometHashAggregate (65) + : +- CometProject (64) + : +- CometBroadcastHashJoin (63) + : :- CometProject (61) + : : +- CometBroadcastHashJoin (60) + : : :- CometProject (58) + : : : +- CometBroadcastHashJoin (57) + : : : :- CometProject (52) + : : : : +- CometBroadcastHashJoin (51) + : : : : :- CometProject (49) + : : : : : +- CometSortMergeJoin (48) + : : : : : :- CometSort (42) + : : : : : : +- CometExchange (41) + : : : : : : +- CometFilter (40) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) + : : : : : +- CometSort (47) + : : : : : +- CometExchange (46) + : : : : : +- CometProject (45) + : : : : : +- CometFilter (44) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : : : +- ReusedExchange (50) + : : : +- CometBroadcastExchange (56) + : : : +- CometProject (55) + : : : +- CometFilter (54) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) + : : +- ReusedExchange (59) + : +- ReusedExchange (62) + +- CometHashAggregate (96) + +- CometExchange (95) + +- CometHashAggregate (94) + +- CometProject (93) + +- CometBroadcastHashJoin (92) + :- CometProject (90) + : +- CometBroadcastHashJoin (89) + : :- CometProject (87) + : : +- CometBroadcastHashJoin (86) + : : :- CometProject (81) + : : : +- CometBroadcastHashJoin (80) + : : : :- CometProject (78) + : : : : +- CometSortMergeJoin (77) + : : : : :- CometSort (71) + : : : : : +- CometExchange (70) + : : : : : +- CometFilter (69) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) + : : : : +- CometSort (76) + : : : : +- CometExchange (75) + : : : : +- CometProject (74) + : : : : +- CometFilter (73) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) + : : : +- ReusedExchange (79) + : : +- CometBroadcastExchange (85) + : : +- CometProject (84) + : : +- CometFilter (83) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) + : +- ReusedExchange (88) + +- ReusedExchange (91) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -280,351 +277,335 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(38) CometHashAggregate Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Keys [1]: [s_store_id#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#33, store channel AS channel#34, concat(store, s_store_id#18) AS id#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) -(42) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(41) CometExchange +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 ASC NULLS FIRST] +(42) CometSort +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(44) CometFilter +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) -(46) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(45) CometProject +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(46) CometExchange +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 ASC NULLS FIRST] +(47) CometSort +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], LeftOuter +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(49) CometProject +Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(52) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#51, 16, true, false, true) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(55) CometProject +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(56) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(57) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(58) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(60) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [i_item_sk#45] +Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(61) CometProject +Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] +Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(63) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [p_promo_sk#46] +Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(64) CometProject +Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] +Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(65) CometHashAggregate +Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(66) CometExchange +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#63, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#64, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#65, catalog channel AS channel#66, concat(catalog_page, cp_catalog_page_id#52) AS id#67] +(67) CometHashAggregate +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(69) CometFilter +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(70) CometExchange +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 ASC NULLS FIRST] +(71) CometSort +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(73) CometFilter +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(74) CometProject +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(75) CometExchange +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(76) CometSort +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], LeftOuter +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(78) CometProject +Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +Right output [1]: [d_date_sk#65] +Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(81) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#66, web_site_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#83, 16, true, false, true) AS web_site_id#84] +(84) CometProject +Input [2]: [web_site_sk#66, web_site_id#67] +Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +Right output [2]: [web_site_sk#66, web_site_id#68] +Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(87) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] +Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [i_item_sk#69] +Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(90) CometProject +Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] +Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(92) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [p_promo_sk#70] +Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(93) CometProject +Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] +Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(94) CometHashAggregate +Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Keys [1]: [web_site_id#68] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(95) CometExchange +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] (96) CometHashAggregate -Input [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#95, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#96, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#97, web channel AS channel#98, concat(web_site, web_site_id#84) AS id#99] - -(100) Union - -(101) Expand [codegen id : 4] -Input [5]: [sales#31, returns#32, profit#33, channel#34, id#35] -Arguments: [[sales#31, returns#32, profit#33, channel#34, id#35, 0], [sales#31, returns#32, profit#33, channel#34, null, 1], [sales#31, returns#32, profit#33, null, null, 3]], [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] - -(102) HashAggregate [codegen id : 4] -Input [6]: [sales#31, returns#32, profit#33, channel#100, id#101, spark_grouping_id#102] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [partial_sum(sales#31), partial_sum(returns#32), partial_sum(profit#33)] -Aggregate Attributes [6]: [sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, isEmpty#108] -Results [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(103) CometColumnarExchange -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Arguments: hashpartitioning(channel#100, id#101, spark_grouping_id#102, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(104) CometColumnarToRow [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] - -(105) HashAggregate [codegen id : 5] -Input [9]: [channel#100, id#101, spark_grouping_id#102, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] -Keys [3]: [channel#100, id#101, spark_grouping_id#102] -Functions [3]: [sum(sales#31), sum(returns#32), sum(profit#33)] -Aggregate Attributes [3]: [sum(sales#31)#115, sum(returns#32)#116, sum(profit#33)#117] -Results [5]: [channel#100, id#101, sum(sales#31)#115 AS sales#118, sum(returns#32)#116 AS returns#119, sum(profit#33)#117 AS profit#120] - -(106) TakeOrderedAndProject -Input [5]: [channel#100, id#101, sales#118, returns#119, profit#120] -Arguments: 100, [channel#100 ASC NULLS FIRST, id#101 ASC NULLS FIRST], [channel#100, id#101, sales#118, returns#119, profit#120] +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Keys [1]: [web_site_id#68] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(97) CometUnion +Child 0 Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] +Child 1 Input [5]: [sales#81, returns#82, profit#83, channel#84, id#85] +Child 2 Input [5]: [sales#86, returns#87, profit#88, channel#89, id#90] + +(98) CometExpand +Input [5]: [sales#76, returns#77, profit#78, channel#79, id#80] +Arguments: [[sales#76, returns#77, profit#78, channel#79, id#80, 0], [sales#76, returns#77, profit#78, channel#79, null, 1], [sales#76, returns#77, profit#78, null, null, 3]], [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] + +(99) CometHashAggregate +Input [6]: [sales#76, returns#77, profit#78, channel#91, id#92, spark_grouping_id#93] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [partial_sum(sales#76), partial_sum(returns#77), partial_sum(profit#78)] + +(100) CometExchange +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Arguments: hashpartitioning(channel#91, id#92, spark_grouping_id#93, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(101) CometHashAggregate +Input [9]: [channel#91, id#92, spark_grouping_id#93, sum#94, isEmpty#95, sum#96, isEmpty#97, sum#98, isEmpty#99] +Keys [3]: [channel#91, id#92, spark_grouping_id#93] +Functions [3]: [sum(sales#76), sum(returns#77), sum(profit#78)] + +(102) CometTakeOrderedAndProject +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#91 ASC NULLS FIRST,id#92 ASC NULLS FIRST], output=[channel#91,id#92,sales#100,returns#101,profit#102]), [channel#91, id#92, sales#100, returns#101, profit#102], 100, 0, [channel#91 ASC NULLS FIRST, id#92 ASC NULLS FIRST], [channel#91, id#92, sales#100, returns#101, profit#102] + +(103) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#91, id#92, sales#100, returns#101, profit#102] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (111) -+- * CometColumnarToRow (110) - +- CometProject (109) - +- CometFilter (108) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (107) +BroadcastExchange (108) ++- * CometColumnarToRow (107) + +- CometProject (106) + +- CometFilter (105) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (104) -(107) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(104) 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,2000-08-23), LessThanOrEqual(d_date,2000-09-22), IsNotNull(d_date_sk)] ReadSchema: struct -(108) CometFilter +(105) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 2000-08-23)) AND (d_date#15 <= 2000-09-22)) AND isnotnull(d_date_sk#14)) -(109) CometProject +(106) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(110) CometColumnarToRow [codegen id : 1] +(107) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(111) BroadcastExchange +(108) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=11] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt index 2c56b98787..198646e0fd 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt @@ -1,126 +1,115 @@ -TakeOrderedAndProject [channel,id,sales,returns,profit] - WholeStageCodegen (5) - HashAggregate [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id,spark_grouping_id] #1 - WholeStageCodegen (4) - HashAggregate [channel,id,spark_grouping_id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - Expand [sales,returns,profit,channel,id] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #2 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #3 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #5 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - 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] - CometBroadcastExchange [s_store_sk,s_store_id] #7 - 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] #8 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #9 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #10 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #11 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #12 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #14 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #15 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #16 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #6 - CometBroadcastExchange [web_site_sk,web_site_id] #17 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #8 - ReusedExchange [p_promo_sk] #9 +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [channel,id,sales,returns,profit] + CometHashAggregate [spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id,spark_grouping_id] #1 + CometHashAggregate [sales,returns,profit] [channel,id,spark_grouping_id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometExpand [channel,id] [sales,returns,profit,channel,id,spark_grouping_id] + CometUnion [sales,returns,profit,channel,id] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),s_store_id] + CometExchange [s_store_id] #2 + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #3 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #5 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + 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] + CometBroadcastExchange [s_store_sk,s_store_id] #7 + 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] #8 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #9 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),cp_catalog_page_id] + CometExchange [cp_catalog_page_id] #10 + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #11 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #12 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #13 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [sales,returns,profit,channel,id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),web_site_id] + CometExchange [web_site_id] #14 + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #15 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #16 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #6 + CometBroadcastExchange [web_site_sk,web_site_id] #17 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #8 + ReusedExchange [p_promo_sk] #9 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/explain.txt index 45c375a787..5bafdc5586 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * Project (32) - +- * CometColumnarToRow (31) +* CometColumnarToRow (33) ++- CometTakeOrderedAndProject (32) + +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometBroadcastExchange (26) : +- CometProject (25) @@ -177,14 +177,14 @@ Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] Right output [1]: [sr_cdemo_sk#18] Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft -(31) CometColumnarToRow [codegen id : 1] +(31) CometProject Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] +Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -(32) Project [codegen id : 1] -Output [3]: [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] - -(33) TakeOrderedAndProject +(32) CometTakeOrderedAndProject Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: 100, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] + +(33) CometColumnarToRow [codegen id : 1] +Input [2]: [customer_id#20, customername#21] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/simplified.txt index cc7cf901e5..fd8d1864ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84.native_iceberg_compat/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [c_customer_id,customer_id,customername] - WholeStageCodegen (1) - Project [c_customer_id,c_last_name,c_first_name] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customername,c_customer_id] + CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt index 45c375a787..5bafdc5586 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/explain.txt @@ -1,7 +1,7 @@ == Physical Plan == -TakeOrderedAndProject (33) -+- * Project (32) - +- * CometColumnarToRow (31) +* CometColumnarToRow (33) ++- CometTakeOrderedAndProject (32) + +- CometProject (31) +- CometBroadcastHashJoin (30) :- CometBroadcastExchange (26) : +- CometProject (25) @@ -177,14 +177,14 @@ Left output [4]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12] Right output [1]: [sr_cdemo_sk#18] Arguments: [cd_demo_sk#12], [sr_cdemo_sk#18], Inner, BuildLeft -(31) CometColumnarToRow [codegen id : 1] +(31) CometProject Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] +Arguments: [customer_id#20, customername#21, c_customer_id#7], [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -(32) Project [codegen id : 1] -Output [3]: [c_customer_id#7 AS customer_id#20, concat(c_last_name#9, , , c_first_name#8) AS customername#21, c_customer_id#7] -Input [5]: [c_customer_id#7, c_first_name#8, c_last_name#9, cd_demo_sk#12, sr_cdemo_sk#18] - -(33) TakeOrderedAndProject +(32) CometTakeOrderedAndProject Input [3]: [customer_id#20, customername#21, c_customer_id#7] -Arguments: 100, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[c_customer_id#7 ASC NULLS FIRST], output=[customer_id#20,customername#21]), [customer_id#20, customername#21], 100, 0, [c_customer_id#7 ASC NULLS FIRST], [customer_id#20, customername#21] + +(33) CometColumnarToRow [codegen id : 1] +Input [2]: [customer_id#20, customername#21] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt index cc7cf901e5..fd8d1864ed 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q84/simplified.txt @@ -1,8 +1,8 @@ -TakeOrderedAndProject [c_customer_id,customer_id,customername] - WholeStageCodegen (1) - Project [c_customer_id,c_last_name,c_first_name] - CometColumnarToRow - InputAdapter +WholeStageCodegen (1) + CometColumnarToRow + InputAdapter + CometTakeOrderedAndProject [customer_id,customername,c_customer_id] + CometProject [c_last_name,c_first_name] [customer_id,customername,c_customer_id] CometBroadcastHashJoin [c_customer_id,c_first_name,c_last_name,cd_demo_sk,sr_cdemo_sk] CometBroadcastExchange [c_customer_id,c_first_name,c_last_name,cd_demo_sk] #1 CometProject [c_customer_id,c_first_name,c_last_name,cd_demo_sk] diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/explain.txt index 0ada41413b..306485ca63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/explain.txt @@ -1,96 +1,88 @@ == Physical Plan == -* CometColumnarToRow (92) -+- CometTakeOrderedAndProject (91) - +- CometHashAggregate (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- Union (87) - :- * HashAggregate (72) - : +- * CometColumnarToRow (71) - : +- CometColumnarExchange (70) - : +- * HashAggregate (69) - : +- Union (68) - : :- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- * HashAggregate (43) - : : +- * CometColumnarToRow (42) - : : +- CometExchange (41) - : : +- CometHashAggregate (40) - : : +- CometProject (39) - : : +- CometBroadcastHashJoin (38) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometUnion (30) - : : : : :- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : : +- CometProject (29) - : : : : +- CometFilter (28) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : : +- ReusedExchange (31) - : : +- CometBroadcastExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometUnion (54) - : : : :- CometProject (46) - : : : : +- CometFilter (45) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : : +- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastExchange (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : : +- CometProject (51) - : : : +- CometFilter (50) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : : +- ReusedExchange (55) - : +- CometBroadcastExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) - :- * HashAggregate (79) - : +- * CometColumnarToRow (78) - : +- CometColumnarExchange (77) - : +- * HashAggregate (76) - : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) - : +- ReusedExchange (73) - +- * HashAggregate (86) - +- * CometColumnarToRow (85) - +- CometColumnarExchange (84) - +- * HashAggregate (83) - +- * HashAggregate (82) - +- * CometColumnarToRow (81) - +- ReusedExchange (80) +* CometColumnarToRow (84) ++- CometTakeOrderedAndProject (83) + +- CometHashAggregate (82) + +- CometExchange (81) + +- CometHashAggregate (80) + +- CometUnion (79) + :- CometHashAggregate (68) + : +- CometExchange (67) + : +- CometHashAggregate (66) + : +- CometUnion (65) + : :- CometHashAggregate (22) + : : +- CometExchange (21) + : : +- CometHashAggregate (20) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometProject (10) + : : : +- CometFilter (9) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (17) + : : +- CometProject (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : :- CometHashAggregate (41) + : : +- CometExchange (40) + : : +- CometHashAggregate (39) + : : +- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (30) + : : +- CometBroadcastExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) + : +- CometHashAggregate (64) + : +- CometExchange (63) + : +- CometHashAggregate (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometUnion (52) + : : : :- CometProject (44) + : : : : +- CometFilter (43) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : : +- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometBroadcastExchange (46) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) + : : : +- CometProject (49) + : : : +- CometFilter (48) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (53) + : +- CometBroadcastExchange (59) + : +- CometProject (58) + : +- CometFilter (57) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) + :- CometHashAggregate (73) + : +- CometExchange (72) + : +- CometHashAggregate (71) + : +- CometHashAggregate (70) + : +- ReusedExchange (69) + +- CometHashAggregate (78) + +- CometExchange (77) + +- CometHashAggregate (76) + +- CometHashAggregate (75) + +- ReusedExchange (74) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -194,381 +186,339 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Keys [1]: [s_store_id#26] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [store channel AS channel#35, concat(store, s_store_id#26) AS id#36, MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : isnotnull(cs_catalog_page_sk#31) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(25) CometProject +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Condition : isnotnull(cr_catalog_page_sk#42) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(28) CometProject +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(29) CometUnion +Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [1]: [d_date_sk#52] +Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight -(33) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(32) CometProject +Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] +Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#63, 16, true, false, true) AS cp_catalog_page_id#64] +(35) CometProject +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(36) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(37) CometBroadcastHashJoin +Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(38) CometProject +Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(39) CometHashAggregate +Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometExchange +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(41) CometHashAggregate +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#64) AS id#74, MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#75, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#76, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(43) CometFilter +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_web_site_sk#60) -(46) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(44) CometProject +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(48) CometFilter +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(49) CometProject +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#103] +(51) CometProject +Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [1]: [d_date_sk#86] +Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(55) CometProject +Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] +Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#87, web_site_id#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(57) CometFilter +Input [2]: [web_site_sk#87, web_site_id#88] +Condition : isnotnull(web_site_sk#87) + +(58) CometProject +Input [2]: [web_site_sk#87, web_site_id#88] +Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] -(60) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#106, 16, true, false, true) AS web_site_id#107] +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] -(61) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(60) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [2]: [web_site_sk#87, web_site_id#89] +Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight -(62) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], Inner, BuildRight +(61) CometProject +Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] +Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -(63) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(62) CometHashAggregate +Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] +Keys [1]: [web_site_id#89] +Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] + +(63) CometExchange +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (64) CometHashAggregate -Input [5]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [web channel AS channel#116, concat(web_site, web_site_id#107) AS id#117, MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#118, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#119, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#120] - -(68) Union - -(69) HashAggregate [codegen id : 4] -Input [5]: [channel#35, id#36, sales#37, returns#38, profit#39] -Keys [2]: [channel#35, id#36] -Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Results [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(70) CometColumnarExchange -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: hashpartitioning(channel#35, id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(71) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(72) HashAggregate [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [5]: [channel#35, id#36, cast(sum(sales#37)#133 as decimal(37,2)) AS sales#136, cast(sum(returns#38)#134 as decimal(37,2)) AS returns#137, cast(sum(profit#39)#135 as decimal(38,2)) AS profit#138] - -(73) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(74) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(75) HashAggregate [codegen id : 10] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [4]: [channel#35, sum(sales#37)#133 AS sales#139, sum(returns#38)#134 AS returns#140, sum(profit#39)#135 AS profit#141] - -(76) HashAggregate [codegen id : 10] -Input [4]: [channel#35, sales#139, returns#140, profit#141] -Keys [1]: [channel#35] -Functions [3]: [partial_sum(sales#139), partial_sum(returns#140), partial_sum(profit#141)] -Aggregate Attributes [6]: [sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Results [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] - -(77) CometColumnarExchange -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Arguments: hashpartitioning(channel#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(78) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] - -(79) HashAggregate [codegen id : 11] -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Keys [1]: [channel#35] -Functions [3]: [sum(sales#139), sum(returns#140), sum(profit#141)] -Aggregate Attributes [3]: [sum(sales#139)#154, sum(returns#140)#155, sum(profit#141)#156] -Results [5]: [channel#35, null AS id#157, sum(sales#139)#154 AS sum(sales)#158, sum(returns#140)#155 AS sum(returns)#159, sum(profit#141)#156 AS sum(profit)#160] - -(80) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(81) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(82) HashAggregate [codegen id : 16] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [3]: [sum(sales#37)#133 AS sales#139, sum(returns#38)#134 AS returns#140, sum(profit#39)#135 AS profit#141] - -(83) HashAggregate [codegen id : 16] -Input [3]: [sales#139, returns#140, profit#141] +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Keys [1]: [web_site_id#89] +Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] + +(65) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] +Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] + +(66) CometHashAggregate +Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Keys [2]: [channel#94, id#95] +Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] + +(67) CometExchange +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(68) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(69) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(70) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(71) CometHashAggregate +Input [4]: [channel#94, sales#115, returns#116, profit#117] +Keys [1]: [channel#94] +Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] + +(72) CometExchange +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(73) CometHashAggregate +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Keys [1]: [channel#94] +Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] + +(74) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(75) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(76) CometHashAggregate +Input [3]: [sales#115, returns#116, profit#117] Keys: [] -Functions [3]: [partial_sum(sales#139), partial_sum(returns#140), partial_sum(profit#141)] -Aggregate Attributes [6]: [sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166] -Results [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] - -(84) CometColumnarExchange -Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] -(85) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] +(77) CometExchange +Input [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(86) HashAggregate [codegen id : 17] -Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] +(78) CometHashAggregate +Input [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] Keys: [] -Functions [3]: [sum(sales#139), sum(returns#140), sum(profit#141)] -Aggregate Attributes [3]: [sum(sales#139)#173, sum(returns#140)#174, sum(profit#141)#175] -Results [5]: [null AS channel#176, null AS id#177, sum(sales#139)#173 AS sum(sales)#178, sum(returns#140)#174 AS sum(returns)#179, sum(profit#141)#175 AS sum(profit)#180] +Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] -(87) Union +(79) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] +Child 1 Input [5]: [channel#94, id#133, sum(sales)#134, sum(returns)#135, sum(profit)#136] +Child 2 Input [5]: [channel#137, id#138, sum(sales)#139, sum(returns)#140, sum(profit)#141] -(88) HashAggregate [codegen id : 18] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(80) CometHashAggregate +Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] +Keys [5]: [channel#94, id#95, sales#130, returns#131, profit#132] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -(89) CometColumnarExchange -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: hashpartitioning(channel#35, id#36, sales#136, returns#137, profit#138, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(81) CometExchange +Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] +Arguments: hashpartitioning(channel#94, id#95, sales#130, returns#131, profit#132, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(90) CometHashAggregate -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(82) CometHashAggregate +Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] +Keys [5]: [channel#94, id#95, sales#130, returns#131, profit#132] Functions: [] -(91) CometTakeOrderedAndProject -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,id#36 ASC NULLS FIRST], output=[channel#35,id#36,sales#136,returns#137,profit#138]), [channel#35, id#36, sales#136, returns#137, profit#138], 100, 0, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35, id#36, sales#136, returns#137, profit#138] +(83) CometTakeOrderedAndProject +Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#130,returns#131,profit#132]), [channel#94, id#95, sales#130, returns#131, profit#132], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#130, returns#131, profit#132] -(92) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(84) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (97) -+- * CometColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (93) +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(94) CometFilter +(86) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(95) CometProject +(87) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(96) CometColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(97) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/simplified.txt index f97451c0c9..c0b236a57a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.native_iceberg_compat/simplified.txt @@ -1,127 +1,99 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [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_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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - 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] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - 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] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (11) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #12 - WholeStageCodegen (10) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (16) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [s_store_id] #3 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [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_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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + 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] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + 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] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [cp_catalog_page_id] #7 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [web_site_id] #9 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #12 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange #13 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt index 0ada41413b..306485ca63 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt @@ -1,96 +1,88 @@ == Physical Plan == -* CometColumnarToRow (92) -+- CometTakeOrderedAndProject (91) - +- CometHashAggregate (90) - +- CometColumnarExchange (89) - +- * HashAggregate (88) - +- Union (87) - :- * HashAggregate (72) - : +- * CometColumnarToRow (71) - : +- CometColumnarExchange (70) - : +- * HashAggregate (69) - : +- Union (68) - : :- * HashAggregate (23) - : : +- * CometColumnarToRow (22) - : : +- CometExchange (21) - : : +- CometHashAggregate (20) - : : +- CometProject (19) - : : +- CometBroadcastHashJoin (18) - : : :- CometProject (13) - : : : +- CometBroadcastHashJoin (12) - : : : :- CometUnion (7) - : : : : :- CometProject (3) - : : : : : +- CometFilter (2) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : +- CometProject (6) - : : : : +- CometFilter (5) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) - : : : +- CometBroadcastExchange (11) - : : : +- CometProject (10) - : : : +- CometFilter (9) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) - : : +- CometBroadcastExchange (17) - : : +- CometProject (16) - : : +- CometFilter (15) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) - : :- * HashAggregate (43) - : : +- * CometColumnarToRow (42) - : : +- CometExchange (41) - : : +- CometHashAggregate (40) - : : +- CometProject (39) - : : +- CometBroadcastHashJoin (38) - : : :- CometProject (33) - : : : +- CometBroadcastHashJoin (32) - : : : :- CometUnion (30) - : : : : :- CometProject (26) - : : : : : +- CometFilter (25) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (24) - : : : : +- CometProject (29) - : : : : +- CometFilter (28) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (27) - : : : +- ReusedExchange (31) - : : +- CometBroadcastExchange (37) - : : +- CometProject (36) - : : +- CometFilter (35) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (34) - : +- * HashAggregate (67) - : +- * CometColumnarToRow (66) - : +- CometExchange (65) - : +- CometHashAggregate (64) - : +- CometProject (63) - : +- CometBroadcastHashJoin (62) - : :- CometProject (57) - : : +- CometBroadcastHashJoin (56) - : : :- CometUnion (54) - : : : :- CometProject (46) - : : : : +- CometFilter (45) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (44) - : : : +- CometProject (53) - : : : +- CometBroadcastHashJoin (52) - : : : :- CometBroadcastExchange (48) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (47) - : : : +- CometProject (51) - : : : +- CometFilter (50) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (49) - : : +- ReusedExchange (55) - : +- CometBroadcastExchange (61) - : +- CometProject (60) - : +- CometFilter (59) - : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (58) - :- * HashAggregate (79) - : +- * CometColumnarToRow (78) - : +- CometColumnarExchange (77) - : +- * HashAggregate (76) - : +- * HashAggregate (75) - : +- * CometColumnarToRow (74) - : +- ReusedExchange (73) - +- * HashAggregate (86) - +- * CometColumnarToRow (85) - +- CometColumnarExchange (84) - +- * HashAggregate (83) - +- * HashAggregate (82) - +- * CometColumnarToRow (81) - +- ReusedExchange (80) +* CometColumnarToRow (84) ++- CometTakeOrderedAndProject (83) + +- CometHashAggregate (82) + +- CometExchange (81) + +- CometHashAggregate (80) + +- CometUnion (79) + :- CometHashAggregate (68) + : +- CometExchange (67) + : +- CometHashAggregate (66) + : +- CometUnion (65) + : :- CometHashAggregate (22) + : : +- CometExchange (21) + : : +- CometHashAggregate (20) + : : +- CometProject (19) + : : +- CometBroadcastHashJoin (18) + : : :- CometProject (13) + : : : +- CometBroadcastHashJoin (12) + : : : :- CometUnion (7) + : : : : :- CometProject (3) + : : : : : +- CometFilter (2) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : +- CometProject (6) + : : : : +- CometFilter (5) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (4) + : : : +- CometBroadcastExchange (11) + : : : +- CometProject (10) + : : : +- CometFilter (9) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (8) + : : +- CometBroadcastExchange (17) + : : +- CometProject (16) + : : +- CometFilter (15) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (14) + : :- CometHashAggregate (41) + : : +- CometExchange (40) + : : +- CometHashAggregate (39) + : : +- CometProject (38) + : : +- CometBroadcastHashJoin (37) + : : :- CometProject (32) + : : : +- CometBroadcastHashJoin (31) + : : : :- CometUnion (29) + : : : : :- CometProject (25) + : : : : : +- CometFilter (24) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (23) + : : : : +- CometProject (28) + : : : : +- CometFilter (27) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (26) + : : : +- ReusedExchange (30) + : : +- CometBroadcastExchange (36) + : : +- CometProject (35) + : : +- CometFilter (34) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (33) + : +- CometHashAggregate (64) + : +- CometExchange (63) + : +- CometHashAggregate (62) + : +- CometProject (61) + : +- CometBroadcastHashJoin (60) + : :- CometProject (55) + : : +- CometBroadcastHashJoin (54) + : : :- CometUnion (52) + : : : :- CometProject (44) + : : : : +- CometFilter (43) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (42) + : : : +- CometProject (51) + : : : +- CometBroadcastHashJoin (50) + : : : :- CometBroadcastExchange (46) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (45) + : : : +- CometProject (49) + : : : +- CometFilter (48) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (47) + : : +- ReusedExchange (53) + : +- CometBroadcastExchange (59) + : +- CometProject (58) + : +- CometFilter (57) + : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (56) + :- CometHashAggregate (73) + : +- CometExchange (72) + : +- CometHashAggregate (71) + : +- CometHashAggregate (70) + : +- ReusedExchange (69) + +- CometHashAggregate (78) + +- CometExchange (77) + +- CometHashAggregate (76) + +- CometHashAggregate (75) + +- ReusedExchange (74) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -194,381 +186,339 @@ Functions [4]: [partial_sum(UnscaledValue(sales_price#8)), partial_sum(UnscaledV Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Arguments: hashpartitioning(s_store_id#26, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=1] -(22) CometColumnarToRow [codegen id : 1] -Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] - -(23) HashAggregate [codegen id : 1] +(22) CometHashAggregate Input [5]: [s_store_id#26, sum#27, sum#28, sum#29, sum#30] Keys [1]: [s_store_id#26] Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#31, sum(UnscaledValue(return_amt#10))#32, sum(UnscaledValue(profit#9))#33, sum(UnscaledValue(net_loss#11))#34] -Results [5]: [store channel AS channel#35, concat(store, s_store_id#26) AS id#36, MakeDecimal(sum(UnscaledValue(sales_price#8))#31,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#32,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#33,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#34,17,2)) AS profit#39] -(24) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] +(23) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#43), dynamicpruningexpression(cs_sold_date_sk#43 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk)] ReadSchema: struct -(25) CometFilter -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Condition : isnotnull(cs_catalog_page_sk#40) +(24) CometFilter +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : isnotnull(cs_catalog_page_sk#31) -(26) CometProject -Input [4]: [cs_catalog_page_sk#40, cs_ext_sales_price#41, cs_net_profit#42, cs_sold_date_sk#43] -Arguments: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50], [cs_catalog_page_sk#40 AS page_sk#45, cs_sold_date_sk#43 AS date_sk#46, cs_ext_sales_price#41 AS sales_price#47, cs_net_profit#42 AS profit#48, 0.00 AS return_amt#49, 0.00 AS net_loss#50] +(25) CometProject +Input [4]: [cs_catalog_page_sk#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41], [cs_catalog_page_sk#31 AS page_sk#36, cs_sold_date_sk#34 AS date_sk#37, cs_ext_sales_price#32 AS sales_price#38, cs_net_profit#33 AS profit#39, 0.00 AS return_amt#40, 0.00 AS net_loss#41] -(27) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] +(26) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cr_returned_date_sk#54), dynamicpruningexpression(cr_returned_date_sk#54 IN dynamicpruning#44)] +PartitionFilters: [isnotnull(cr_returned_date_sk#45), dynamicpruningexpression(cr_returned_date_sk#45 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cr_catalog_page_sk)] ReadSchema: struct -(28) CometFilter -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Condition : isnotnull(cr_catalog_page_sk#51) +(27) CometFilter +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Condition : isnotnull(cr_catalog_page_sk#42) -(29) CometProject -Input [4]: [cr_catalog_page_sk#51, cr_return_amount#52, cr_net_loss#53, cr_returned_date_sk#54] -Arguments: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60], [cr_catalog_page_sk#51 AS page_sk#55, cr_returned_date_sk#54 AS date_sk#56, 0.00 AS sales_price#57, 0.00 AS profit#58, cr_return_amount#52 AS return_amt#59, cr_net_loss#53 AS net_loss#60] +(28) CometProject +Input [4]: [cr_catalog_page_sk#42, cr_return_amount#43, cr_net_loss#44, cr_returned_date_sk#45] +Arguments: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51], [cr_catalog_page_sk#42 AS page_sk#46, cr_returned_date_sk#45 AS date_sk#47, 0.00 AS sales_price#48, 0.00 AS profit#49, cr_return_amount#43 AS return_amt#50, cr_net_loss#44 AS net_loss#51] -(30) CometUnion -Child 0 Input [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Child 1 Input [6]: [page_sk#55, date_sk#56, sales_price#57, profit#58, return_amt#59, net_loss#60] +(29) CometUnion +Child 0 Input [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Child 1 Input [6]: [page_sk#46, date_sk#47, sales_price#48, profit#49, return_amt#50, net_loss#51] -(31) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#61] +(30) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#52] -(32) CometBroadcastHashJoin -Left output [6]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [1]: [d_date_sk#61] -Arguments: [date_sk#46], [d_date_sk#61], Inner, BuildRight +(31) CometBroadcastHashJoin +Left output [6]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [1]: [d_date_sk#52] +Arguments: [date_sk#37], [d_date_sk#52], Inner, BuildRight -(33) CometProject -Input [7]: [page_sk#45, date_sk#46, sales_price#47, profit#48, return_amt#49, net_loss#50, d_date_sk#61] -Arguments: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50], [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] +(32) CometProject +Input [7]: [page_sk#36, date_sk#37, sales_price#38, profit#39, return_amt#40, net_loss#41, d_date_sk#52] +Arguments: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41], [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] -(34) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] +(33) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(35) CometFilter -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Condition : isnotnull(cp_catalog_page_sk#62) +(34) CometFilter +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Condition : isnotnull(cp_catalog_page_sk#53) -(36) CometProject -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#63] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64], [cp_catalog_page_sk#62, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#63, 16, true, false, true) AS cp_catalog_page_id#64] +(35) CometProject +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#54] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55], [cp_catalog_page_sk#53, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#54, 16, true, false, true) AS cp_catalog_page_id#55] -(37) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [cp_catalog_page_sk#62, cp_catalog_page_id#64] +(36) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [cp_catalog_page_sk#53, cp_catalog_page_id#55] -(38) CometBroadcastHashJoin -Left output [5]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50] -Right output [2]: [cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [page_sk#45], [cp_catalog_page_sk#62], Inner, BuildRight +(37) CometBroadcastHashJoin +Left output [5]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41] +Right output [2]: [cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [page_sk#36], [cp_catalog_page_sk#53], Inner, BuildRight -(39) CometProject -Input [7]: [page_sk#45, sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_sk#62, cp_catalog_page_id#64] -Arguments: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64], [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] +(38) CometProject +Input [7]: [page_sk#36, sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_sk#53, cp_catalog_page_id#55] +Arguments: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55], [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] -(40) CometHashAggregate -Input [5]: [sales_price#47, profit#48, return_amt#49, net_loss#50, cp_catalog_page_id#64] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [partial_sum(UnscaledValue(sales_price#47)), partial_sum(UnscaledValue(return_amt#49)), partial_sum(UnscaledValue(profit#48)), partial_sum(UnscaledValue(net_loss#50))] +(39) CometHashAggregate +Input [5]: [sales_price#38, profit#39, return_amt#40, net_loss#41, cp_catalog_page_id#55] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [partial_sum(UnscaledValue(sales_price#38)), partial_sum(UnscaledValue(return_amt#40)), partial_sum(UnscaledValue(profit#39)), partial_sum(UnscaledValue(net_loss#41))] -(41) CometExchange -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Arguments: hashpartitioning(cp_catalog_page_id#64, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] +(40) CometExchange +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Arguments: hashpartitioning(cp_catalog_page_id#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=2] -(42) CometColumnarToRow [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] +(41) CometHashAggregate +Input [5]: [cp_catalog_page_id#55, sum#56, sum#57, sum#58, sum#59] +Keys [1]: [cp_catalog_page_id#55] +Functions [4]: [sum(UnscaledValue(sales_price#38)), sum(UnscaledValue(return_amt#40)), sum(UnscaledValue(profit#39)), sum(UnscaledValue(net_loss#41))] -(43) HashAggregate [codegen id : 2] -Input [5]: [cp_catalog_page_id#64, sum#65, sum#66, sum#67, sum#68] -Keys [1]: [cp_catalog_page_id#64] -Functions [4]: [sum(UnscaledValue(sales_price#47)), sum(UnscaledValue(return_amt#49)), sum(UnscaledValue(profit#48)), sum(UnscaledValue(net_loss#50))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#47))#69, sum(UnscaledValue(return_amt#49))#70, sum(UnscaledValue(profit#48))#71, sum(UnscaledValue(net_loss#50))#72] -Results [5]: [catalog channel AS channel#73, concat(catalog_page, cp_catalog_page_id#64) AS id#74, MakeDecimal(sum(UnscaledValue(sales_price#47))#69,17,2) AS sales#75, MakeDecimal(sum(UnscaledValue(return_amt#49))#70,17,2) AS returns#76, (MakeDecimal(sum(UnscaledValue(profit#48))#71,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#50))#72,17,2)) AS profit#77] - -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] +(42) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#81), dynamicpruningexpression(ws_sold_date_sk#81 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(ws_sold_date_sk#63), dynamicpruningexpression(ws_sold_date_sk#63 IN dynamicpruning#64)] PushedFilters: [IsNotNull(ws_web_site_sk)] ReadSchema: struct -(45) CometFilter -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Condition : isnotnull(ws_web_site_sk#78) +(43) CometFilter +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Condition : isnotnull(ws_web_site_sk#60) -(46) CometProject -Input [4]: [ws_web_site_sk#78, ws_ext_sales_price#79, ws_net_profit#80, ws_sold_date_sk#81] -Arguments: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88], [ws_web_site_sk#78 AS wsr_web_site_sk#83, ws_sold_date_sk#81 AS date_sk#84, ws_ext_sales_price#79 AS sales_price#85, ws_net_profit#80 AS profit#86, 0.00 AS return_amt#87, 0.00 AS net_loss#88] +(44) CometProject +Input [4]: [ws_web_site_sk#60, ws_ext_sales_price#61, ws_net_profit#62, ws_sold_date_sk#63] +Arguments: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70], [ws_web_site_sk#60 AS wsr_web_site_sk#65, ws_sold_date_sk#63 AS date_sk#66, ws_ext_sales_price#61 AS sales_price#67, ws_net_profit#62 AS profit#68, 0.00 AS return_amt#69, 0.00 AS net_loss#70] -(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(45) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(wr_returned_date_sk#93), dynamicpruningexpression(wr_returned_date_sk#93 IN dynamicpruning#82)] +PartitionFilters: [isnotnull(wr_returned_date_sk#75), dynamicpruningexpression(wr_returned_date_sk#75 IN dynamicpruning#64)] ReadSchema: struct -(48) CometBroadcastExchange -Input [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Arguments: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] +(46) CometBroadcastExchange +Input [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Arguments: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] -(49) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] +(47) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] Batched: true Location [not included in comparison]/{warehouse_dir}/web_sales] PushedFilters: [IsNotNull(ws_item_sk), IsNotNull(ws_order_number), IsNotNull(ws_web_site_sk)] ReadSchema: struct -(50) CometFilter -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Condition : ((isnotnull(ws_item_sk#94) AND isnotnull(ws_order_number#96)) AND isnotnull(ws_web_site_sk#95)) +(48) CometFilter +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Condition : ((isnotnull(ws_item_sk#76) AND isnotnull(ws_order_number#78)) AND isnotnull(ws_web_site_sk#77)) -(51) CometProject -Input [4]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96, ws_sold_date_sk#97] -Arguments: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96], [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] +(49) CometProject +Input [4]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78, ws_sold_date_sk#79] +Arguments: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78], [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] -(52) CometBroadcastHashJoin -Left output [5]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93] -Right output [3]: [ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wr_item_sk#89, wr_order_number#90], [ws_item_sk#94, ws_order_number#96], Inner, BuildLeft +(50) CometBroadcastHashJoin +Left output [5]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75] +Right output [3]: [ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wr_item_sk#71, wr_order_number#72], [ws_item_sk#76, ws_order_number#78], Inner, BuildLeft -(53) CometProject -Input [8]: [wr_item_sk#89, wr_order_number#90, wr_return_amt#91, wr_net_loss#92, wr_returned_date_sk#93, ws_item_sk#94, ws_web_site_sk#95, ws_order_number#96] -Arguments: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103], [ws_web_site_sk#95 AS wsr_web_site_sk#98, wr_returned_date_sk#93 AS date_sk#99, 0.00 AS sales_price#100, 0.00 AS profit#101, wr_return_amt#91 AS return_amt#102, wr_net_loss#92 AS net_loss#103] +(51) CometProject +Input [8]: [wr_item_sk#71, wr_order_number#72, wr_return_amt#73, wr_net_loss#74, wr_returned_date_sk#75, ws_item_sk#76, ws_web_site_sk#77, ws_order_number#78] +Arguments: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85], [ws_web_site_sk#77 AS wsr_web_site_sk#80, wr_returned_date_sk#75 AS date_sk#81, 0.00 AS sales_price#82, 0.00 AS profit#83, wr_return_amt#73 AS return_amt#84, wr_net_loss#74 AS net_loss#85] -(54) CometUnion -Child 0 Input [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Child 1 Input [6]: [wsr_web_site_sk#98, date_sk#99, sales_price#100, profit#101, return_amt#102, net_loss#103] +(52) CometUnion +Child 0 Input [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Child 1 Input [6]: [wsr_web_site_sk#80, date_sk#81, sales_price#82, profit#83, return_amt#84, net_loss#85] -(55) ReusedExchange [Reuses operator id: 11] -Output [1]: [d_date_sk#104] +(53) ReusedExchange [Reuses operator id: 11] +Output [1]: [d_date_sk#86] -(56) CometBroadcastHashJoin -Left output [6]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [1]: [d_date_sk#104] -Arguments: [date_sk#84], [d_date_sk#104], Inner, BuildRight +(54) CometBroadcastHashJoin +Left output [6]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [1]: [d_date_sk#86] +Arguments: [date_sk#66], [d_date_sk#86], Inner, BuildRight -(57) CometProject -Input [7]: [wsr_web_site_sk#83, date_sk#84, sales_price#85, profit#86, return_amt#87, net_loss#88, d_date_sk#104] -Arguments: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88], [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] +(55) CometProject +Input [7]: [wsr_web_site_sk#65, date_sk#66, sales_price#67, profit#68, return_amt#69, net_loss#70, d_date_sk#86] +Arguments: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70], [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] -(58) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#105, web_site_id#106] +(56) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#87, web_site_id#88] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(59) CometFilter -Input [2]: [web_site_sk#105, web_site_id#106] -Condition : isnotnull(web_site_sk#105) +(57) CometFilter +Input [2]: [web_site_sk#87, web_site_id#88] +Condition : isnotnull(web_site_sk#87) + +(58) CometProject +Input [2]: [web_site_sk#87, web_site_id#88] +Arguments: [web_site_sk#87, web_site_id#89], [web_site_sk#87, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#88, 16, true, false, true) AS web_site_id#89] -(60) CometProject -Input [2]: [web_site_sk#105, web_site_id#106] -Arguments: [web_site_sk#105, web_site_id#107], [web_site_sk#105, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#106, 16, true, false, true) AS web_site_id#107] +(59) CometBroadcastExchange +Input [2]: [web_site_sk#87, web_site_id#89] +Arguments: [web_site_sk#87, web_site_id#89] -(61) CometBroadcastExchange -Input [2]: [web_site_sk#105, web_site_id#107] -Arguments: [web_site_sk#105, web_site_id#107] +(60) CometBroadcastHashJoin +Left output [5]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70] +Right output [2]: [web_site_sk#87, web_site_id#89] +Arguments: [wsr_web_site_sk#65], [web_site_sk#87], Inner, BuildRight -(62) CometBroadcastHashJoin -Left output [5]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88] -Right output [2]: [web_site_sk#105, web_site_id#107] -Arguments: [wsr_web_site_sk#83], [web_site_sk#105], Inner, BuildRight +(61) CometProject +Input [7]: [wsr_web_site_sk#65, sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_sk#87, web_site_id#89] +Arguments: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89], [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] -(63) CometProject -Input [7]: [wsr_web_site_sk#83, sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_sk#105, web_site_id#107] -Arguments: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107], [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] +(62) CometHashAggregate +Input [5]: [sales_price#67, profit#68, return_amt#69, net_loss#70, web_site_id#89] +Keys [1]: [web_site_id#89] +Functions [4]: [partial_sum(UnscaledValue(sales_price#67)), partial_sum(UnscaledValue(return_amt#69)), partial_sum(UnscaledValue(profit#68)), partial_sum(UnscaledValue(net_loss#70))] + +(63) CometExchange +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Arguments: hashpartitioning(web_site_id#89, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] (64) CometHashAggregate -Input [5]: [sales_price#85, profit#86, return_amt#87, net_loss#88, web_site_id#107] -Keys [1]: [web_site_id#107] -Functions [4]: [partial_sum(UnscaledValue(sales_price#85)), partial_sum(UnscaledValue(return_amt#87)), partial_sum(UnscaledValue(profit#86)), partial_sum(UnscaledValue(net_loss#88))] - -(65) CometExchange -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Arguments: hashpartitioning(web_site_id#107, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] - -(66) CometColumnarToRow [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] - -(67) HashAggregate [codegen id : 3] -Input [5]: [web_site_id#107, sum#108, sum#109, sum#110, sum#111] -Keys [1]: [web_site_id#107] -Functions [4]: [sum(UnscaledValue(sales_price#85)), sum(UnscaledValue(return_amt#87)), sum(UnscaledValue(profit#86)), sum(UnscaledValue(net_loss#88))] -Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#85))#112, sum(UnscaledValue(return_amt#87))#113, sum(UnscaledValue(profit#86))#114, sum(UnscaledValue(net_loss#88))#115] -Results [5]: [web channel AS channel#116, concat(web_site, web_site_id#107) AS id#117, MakeDecimal(sum(UnscaledValue(sales_price#85))#112,17,2) AS sales#118, MakeDecimal(sum(UnscaledValue(return_amt#87))#113,17,2) AS returns#119, (MakeDecimal(sum(UnscaledValue(profit#86))#114,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#88))#115,17,2)) AS profit#120] - -(68) Union - -(69) HashAggregate [codegen id : 4] -Input [5]: [channel#35, id#36, sales#37, returns#38, profit#39] -Keys [2]: [channel#35, id#36] -Functions [3]: [partial_sum(sales#37), partial_sum(returns#38), partial_sum(profit#39)] -Aggregate Attributes [6]: [sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Results [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(70) CometColumnarExchange -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: hashpartitioning(channel#35, id#36, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=4] - -(71) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(72) HashAggregate [codegen id : 5] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [5]: [channel#35, id#36, cast(sum(sales#37)#133 as decimal(37,2)) AS sales#136, cast(sum(returns#38)#134 as decimal(37,2)) AS returns#137, cast(sum(profit#39)#135 as decimal(38,2)) AS profit#138] - -(73) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(74) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(75) HashAggregate [codegen id : 10] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [4]: [channel#35, sum(sales#37)#133 AS sales#139, sum(returns#38)#134 AS returns#140, sum(profit#39)#135 AS profit#141] - -(76) HashAggregate [codegen id : 10] -Input [4]: [channel#35, sales#139, returns#140, profit#141] -Keys [1]: [channel#35] -Functions [3]: [partial_sum(sales#139), partial_sum(returns#140), partial_sum(profit#141)] -Aggregate Attributes [6]: [sum#142, isEmpty#143, sum#144, isEmpty#145, sum#146, isEmpty#147] -Results [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] - -(77) CometColumnarExchange -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Arguments: hashpartitioning(channel#35, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=5] - -(78) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] - -(79) HashAggregate [codegen id : 11] -Input [7]: [channel#35, sum#148, isEmpty#149, sum#150, isEmpty#151, sum#152, isEmpty#153] -Keys [1]: [channel#35] -Functions [3]: [sum(sales#139), sum(returns#140), sum(profit#141)] -Aggregate Attributes [3]: [sum(sales#139)#154, sum(returns#140)#155, sum(profit#141)#156] -Results [5]: [channel#35, null AS id#157, sum(sales#139)#154 AS sum(sales)#158, sum(returns#140)#155 AS sum(returns)#159, sum(profit#141)#156 AS sum(profit)#160] - -(80) ReusedExchange [Reuses operator id: 70] -Output [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(81) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(82) HashAggregate [codegen id : 16] -Input [8]: [channel#35, id#36, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [2]: [channel#35, id#36] -Functions [3]: [sum(sales#37), sum(returns#38), sum(profit#39)] -Aggregate Attributes [3]: [sum(sales#37)#133, sum(returns#38)#134, sum(profit#39)#135] -Results [3]: [sum(sales#37)#133 AS sales#139, sum(returns#38)#134 AS returns#140, sum(profit#39)#135 AS profit#141] - -(83) HashAggregate [codegen id : 16] -Input [3]: [sales#139, returns#140, profit#141] +Input [5]: [web_site_id#89, sum#90, sum#91, sum#92, sum#93] +Keys [1]: [web_site_id#89] +Functions [4]: [sum(UnscaledValue(sales_price#67)), sum(UnscaledValue(return_amt#69)), sum(UnscaledValue(profit#68)), sum(UnscaledValue(net_loss#70))] + +(65) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Child 1 Input [5]: [channel#99, id#100, sales#101, returns#102, profit#103] +Child 2 Input [5]: [channel#104, id#105, sales#106, returns#107, profit#108] + +(66) CometHashAggregate +Input [5]: [channel#94, id#95, sales#96, returns#97, profit#98] +Keys [2]: [channel#94, id#95] +Functions [3]: [partial_sum(sales#96), partial_sum(returns#97), partial_sum(profit#98)] + +(67) CometExchange +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Arguments: hashpartitioning(channel#94, id#95, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] + +(68) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(69) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(70) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(71) CometHashAggregate +Input [4]: [channel#94, sales#115, returns#116, profit#117] +Keys [1]: [channel#94] +Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] + +(72) CometExchange +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Arguments: hashpartitioning(channel#94, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] + +(73) CometHashAggregate +Input [7]: [channel#94, sum#118, isEmpty#119, sum#120, isEmpty#121, sum#122, isEmpty#123] +Keys [1]: [channel#94] +Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] + +(74) ReusedExchange [Reuses operator id: 67] +Output [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] + +(75) CometHashAggregate +Input [8]: [channel#94, id#95, sum#109, isEmpty#110, sum#111, isEmpty#112, sum#113, isEmpty#114] +Keys [2]: [channel#94, id#95] +Functions [3]: [sum(sales#96), sum(returns#97), sum(profit#98)] + +(76) CometHashAggregate +Input [3]: [sales#115, returns#116, profit#117] Keys: [] -Functions [3]: [partial_sum(sales#139), partial_sum(returns#140), partial_sum(profit#141)] -Aggregate Attributes [6]: [sum#161, isEmpty#162, sum#163, isEmpty#164, sum#165, isEmpty#166] -Results [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] - -(84) CometColumnarExchange -Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=6] +Functions [3]: [partial_sum(sales#115), partial_sum(returns#116), partial_sum(profit#117)] -(85) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] +(77) CometExchange +Input [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(86) HashAggregate [codegen id : 17] -Input [6]: [sum#167, isEmpty#168, sum#169, isEmpty#170, sum#171, isEmpty#172] +(78) CometHashAggregate +Input [6]: [sum#124, isEmpty#125, sum#126, isEmpty#127, sum#128, isEmpty#129] Keys: [] -Functions [3]: [sum(sales#139), sum(returns#140), sum(profit#141)] -Aggregate Attributes [3]: [sum(sales#139)#173, sum(returns#140)#174, sum(profit#141)#175] -Results [5]: [null AS channel#176, null AS id#177, sum(sales#139)#173 AS sum(sales)#178, sum(returns#140)#174 AS sum(returns)#179, sum(profit#141)#175 AS sum(profit)#180] +Functions [3]: [sum(sales#115), sum(returns#116), sum(profit#117)] -(87) Union +(79) CometUnion +Child 0 Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] +Child 1 Input [5]: [channel#94, id#133, sum(sales)#134, sum(returns)#135, sum(profit)#136] +Child 2 Input [5]: [channel#137, id#138, sum(sales)#139, sum(returns)#140, sum(profit)#141] -(88) HashAggregate [codegen id : 18] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(80) CometHashAggregate +Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] +Keys [5]: [channel#94, id#95, sales#130, returns#131, profit#132] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -(89) CometColumnarExchange -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: hashpartitioning(channel#35, id#36, sales#136, returns#137, profit#138, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=7] +(81) CometExchange +Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] +Arguments: hashpartitioning(channel#94, id#95, sales#130, returns#131, profit#132, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(90) CometHashAggregate -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Keys [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(82) CometHashAggregate +Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] +Keys [5]: [channel#94, id#95, sales#130, returns#131, profit#132] Functions: [] -(91) CometTakeOrderedAndProject -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#35 ASC NULLS FIRST,id#36 ASC NULLS FIRST], output=[channel#35,id#36,sales#136,returns#137,profit#138]), [channel#35, id#36, sales#136, returns#137, profit#138], 100, 0, [channel#35 ASC NULLS FIRST, id#36 ASC NULLS FIRST], [channel#35, id#36, sales#136, returns#137, profit#138] +(83) CometTakeOrderedAndProject +Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#94 ASC NULLS FIRST,id#95 ASC NULLS FIRST], output=[channel#94,id#95,sales#130,returns#131,profit#132]), [channel#94, id#95, sales#130, returns#131, profit#132], 100, 0, [channel#94 ASC NULLS FIRST, id#95 ASC NULLS FIRST], [channel#94, id#95, sales#130, returns#131, profit#132] -(92) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#35, id#36, sales#136, returns#137, profit#138] +(84) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#94, id#95, sales#130, returns#131, profit#132] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5 -BroadcastExchange (97) -+- * CometColumnarToRow (96) - +- CometProject (95) - +- CometFilter (94) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (93) +BroadcastExchange (89) ++- * CometColumnarToRow (88) + +- CometProject (87) + +- CometFilter (86) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (85) -(93) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(85) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim Output [2]: [d_date_sk#22, d_date#23] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_date), GreaterThanOrEqual(d_date,1998-08-04), LessThanOrEqual(d_date,1998-08-18), IsNotNull(d_date_sk)] ReadSchema: struct -(94) CometFilter +(86) CometFilter Input [2]: [d_date_sk#22, d_date#23] Condition : (((isnotnull(d_date#23) AND (d_date#23 >= 1998-08-04)) AND (d_date#23 <= 1998-08-18)) AND isnotnull(d_date_sk#22)) -(95) CometProject +(87) CometProject Input [2]: [d_date_sk#22, d_date#23] Arguments: [d_date_sk#22], [d_date_sk#22] -(96) CometColumnarToRow [codegen id : 1] +(88) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#22] -(97) BroadcastExchange +(89) BroadcastExchange Input [1]: [d_date_sk#22] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8] Subquery:2 Hosting operator id = 4 Hosting Expression = sr_returned_date_sk#15 IN dynamicpruning#5 -Subquery:3 Hosting operator id = 24 Hosting Expression = cs_sold_date_sk#43 IN dynamicpruning#5 +Subquery:3 Hosting operator id = 23 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#5 -Subquery:4 Hosting operator id = 27 Hosting Expression = cr_returned_date_sk#54 IN dynamicpruning#5 +Subquery:4 Hosting operator id = 26 Hosting Expression = cr_returned_date_sk#45 IN dynamicpruning#5 -Subquery:5 Hosting operator id = 44 Hosting Expression = ws_sold_date_sk#81 IN dynamicpruning#5 +Subquery:5 Hosting operator id = 42 Hosting Expression = ws_sold_date_sk#63 IN dynamicpruning#5 -Subquery:6 Hosting operator id = 47 Hosting Expression = wr_returned_date_sk#93 IN dynamicpruning#5 +Subquery:6 Hosting operator id = 45 Hosting Expression = wr_returned_date_sk#75 IN dynamicpruning#5 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt index f97451c0c9..c0b236a57a 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/simplified.txt @@ -1,127 +1,99 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #3 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,s_store_id] - CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] - CometProject [store_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [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_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_date] - CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] - CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - 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] - CometBroadcastExchange [s_store_sk,s_store_id] #6 - 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] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #7 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [page_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,sum,sum] [sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss)),channel,id,sales,returns,profit,sum,sum,sum,sum] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #9 - CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] - CometProject [sales_price,profit,return_amt,net_loss,web_site_id] - CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] - CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] - CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] - CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] - CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedSubquery [d_date_sk] #1 - CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] - CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] - ReusedExchange [d_date_sk] #5 - CometBroadcastExchange [web_site_sk,web_site_id] #11 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - WholeStageCodegen (11) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #12 - WholeStageCodegen (10) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sum(sales),sum(returns),sum(profit),sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #13 - WholeStageCodegen (16) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [s_store_id] #3 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [s_store_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,s_store_id] + CometBroadcastHashJoin [store_sk,sales_price,profit,return_amt,net_loss,s_store_sk,s_store_id] + CometProject [store_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [store_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ss_store_sk,ss_sold_date_sk,ss_ext_sales_price,ss_net_profit] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [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_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_date] + CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim [d_date_sk,d_date] + CometProject [sr_store_sk,sr_returned_date_sk,sr_return_amt,sr_net_loss] [store_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_store_sk,sr_return_amt,sr_net_loss,sr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + 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] + CometBroadcastExchange [s_store_sk,s_store_id] #6 + 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] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [cp_catalog_page_id] #7 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [cp_catalog_page_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [page_sk,sales_price,profit,return_amt,net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [page_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [page_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [cs_catalog_page_sk,cs_sold_date_sk,cs_ext_sales_price,cs_net_profit] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [cr_catalog_page_sk,cr_returned_date_sk,cr_return_amount,cr_net_loss] [page_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_catalog_page_sk,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #8 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + CometHashAggregate [sum,sum,sum,sum] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(sales_price)),sum(UnscaledValue(return_amt)),sum(UnscaledValue(profit)),sum(UnscaledValue(net_loss))] + CometExchange [web_site_id] #9 + CometHashAggregate [sales_price,return_amt,profit,net_loss] [web_site_id,sum,sum,sum,sum] + CometProject [sales_price,profit,return_amt,net_loss,web_site_id] + CometBroadcastHashJoin [wsr_web_site_sk,sales_price,profit,return_amt,net_loss,web_site_sk,web_site_id] + CometProject [wsr_web_site_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss,d_date_sk] + CometUnion [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometProject [ws_web_site_sk,ws_sold_date_sk,ws_ext_sales_price,ws_net_profit] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometFilter [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_web_site_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_web_site_sk,wr_returned_date_sk,wr_return_amt,wr_net_loss] [wsr_web_site_sk,date_sk,sales_price,profit,return_amt,net_loss] + CometBroadcastHashJoin [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk,ws_item_sk,ws_web_site_sk,ws_order_number] + CometBroadcastExchange [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] #10 + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedSubquery [d_date_sk] #1 + CometProject [ws_item_sk,ws_web_site_sk,ws_order_number] + CometFilter [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_order_number,ws_sold_date_sk] + ReusedExchange [d_date_sk] #5 + CometBroadcastExchange [web_site_sk,web_site_id] #11 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #12 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sum(sales),sum(returns),sum(profit),sum(sales),sum(returns),sum(profit)] + CometExchange #13 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/explain.txt index d6b731eb98..6898d458e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/explain.txt @@ -1,128 +1,120 @@ == Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometHashAggregate (122) - +- CometColumnarExchange (121) - +- * HashAggregate (120) - +- Union (119) - :- * HashAggregate (104) - : +- * CometColumnarToRow (103) - : +- CometColumnarExchange (102) - : +- * HashAggregate (101) - : +- Union (100) - : :- * HashAggregate (39) - : : +- * CometColumnarToRow (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) - : : +- CometExchange (67) - : : +- CometHashAggregate (66) - : : +- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (62) - : : : +- CometBroadcastHashJoin (61) - : : : :- CometProject (59) - : : : : +- CometBroadcastHashJoin (58) - : : : : :- CometProject (53) - : : : : : +- CometBroadcastHashJoin (52) - : : : : : :- CometProject (50) - : : : : : : +- CometSortMergeJoin (49) - : : : : : : :- CometSort (43) - : : : : : : : +- CometExchange (42) - : : : : : : : +- CometFilter (41) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- CometSort (48) - : : : : : : +- CometExchange (47) - : : : : : : +- CometProject (46) - : : : : : : +- CometFilter (45) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : : +- ReusedExchange (51) - : : : : +- CometBroadcastExchange (57) - : : : : +- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : : +- ReusedExchange (60) - : : +- ReusedExchange (63) - : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) - : +- CometExchange (97) - : +- CometHashAggregate (96) - : +- CometProject (95) - : +- CometBroadcastHashJoin (94) - : :- CometProject (92) - : : +- CometBroadcastHashJoin (91) - : : :- CometProject (89) - : : : +- CometBroadcastHashJoin (88) - : : : :- CometProject (83) - : : : : +- CometBroadcastHashJoin (82) - : : : : :- CometProject (80) - : : : : : +- CometSortMergeJoin (79) - : : : : : :- CometSort (73) - : : : : : : +- CometExchange (72) - : : : : : : +- CometFilter (71) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : : +- CometSort (78) - : : : : : +- CometExchange (77) - : : : : : +- CometProject (76) - : : : : : +- CometFilter (75) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : : +- ReusedExchange (81) - : : : +- CometBroadcastExchange (87) - : : : +- CometProject (86) - : : : +- CometFilter (85) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (111) - : +- * CometColumnarToRow (110) - : +- CometColumnarExchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * CometColumnarToRow (106) - : +- ReusedExchange (105) - +- * HashAggregate (118) - +- * CometColumnarToRow (117) - +- CometColumnarExchange (116) - +- * HashAggregate (115) - +- * HashAggregate (114) - +- * CometColumnarToRow (113) - +- ReusedExchange (112) +* CometColumnarToRow (116) ++- CometTakeOrderedAndProject (115) + +- CometHashAggregate (114) + +- CometExchange (113) + +- CometHashAggregate (112) + +- CometUnion (111) + :- CometHashAggregate (100) + : +- CometExchange (99) + : +- CometHashAggregate (98) + : +- CometUnion (97) + : :- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (29) + : : : +- CometBroadcastHashJoin (28) + : : : :- CometProject (23) + : : : : +- CometBroadcastHashJoin (22) + : : : : :- CometProject (17) + : : : : : +- CometBroadcastHashJoin (16) + : : : : : :- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : : +- CometBroadcastExchange (15) + : : : : : +- CometProject (14) + : : : : : +- CometFilter (13) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) + : : : : +- CometBroadcastExchange (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) + : : : +- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + : : +- CometBroadcastExchange (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + : :- CometHashAggregate (67) + : : +- CometExchange (66) + : : +- CometHashAggregate (65) + : : +- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometProject (61) + : : : +- CometBroadcastHashJoin (60) + : : : :- CometProject (58) + : : : : +- CometBroadcastHashJoin (57) + : : : : :- CometProject (52) + : : : : : +- CometBroadcastHashJoin (51) + : : : : : :- CometProject (49) + : : : : : : +- CometSortMergeJoin (48) + : : : : : : :- CometSort (42) + : : : : : : : +- CometExchange (41) + : : : : : : : +- CometFilter (40) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- CometSort (47) + : : : : : : +- CometExchange (46) + : : : : : : +- CometProject (45) + : : : : : : +- CometFilter (44) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : : : : +- ReusedExchange (50) + : : : : +- CometBroadcastExchange (56) + : : : : +- CometProject (55) + : : : : +- CometFilter (54) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) + : : : +- ReusedExchange (59) + : : +- ReusedExchange (62) + : +- CometHashAggregate (96) + : +- CometExchange (95) + : +- CometHashAggregate (94) + : +- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometProject (81) + : : : : +- CometBroadcastHashJoin (80) + : : : : :- CometProject (78) + : : : : : +- CometSortMergeJoin (77) + : : : : : :- CometSort (71) + : : : : : : +- CometExchange (70) + : : : : : : +- CometFilter (69) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) + : : : : : +- CometSort (76) + : : : : : +- CometExchange (75) + : : : : : +- CometProject (74) + : : : : : +- CometFilter (73) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) + : : : : +- ReusedExchange (79) + : : : +- CometBroadcastExchange (85) + : : : +- CometProject (84) + : : : +- CometFilter (83) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) + : : +- ReusedExchange (88) + : +- ReusedExchange (91) + :- CometHashAggregate (105) + : +- CometExchange (104) + : +- CometHashAggregate (103) + : +- CometHashAggregate (102) + : +- ReusedExchange (101) + +- CometHashAggregate (110) + +- CometExchange (109) + +- CometHashAggregate (108) + +- CometHashAggregate (107) + +- ReusedExchange (106) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -298,436 +290,394 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(38) CometHashAggregate Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Keys [1]: [s_store_id#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [store channel AS channel#31, concat(store, s_store_id#18) AS id#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#33, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#34, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) -(42) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(41) CometExchange +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 ASC NULLS FIRST] +(42) CometSort +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(44) CometFilter +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) -(46) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(45) CometProject +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(46) CometExchange +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 ASC NULLS FIRST] +(47) CometSort +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], LeftOuter +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(49) CometProject +Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(52) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#51, 16, true, false, true) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(55) CometProject +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(56) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(57) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(58) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(60) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [i_item_sk#45] +Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(61) CometProject +Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] +Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(63) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [p_promo_sk#46] +Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(64) CometProject +Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] +Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(65) CometHashAggregate +Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(66) CometExchange +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [catalog channel AS channel#63, concat(catalog_page, cp_catalog_page_id#52) AS id#64, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#65, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#66, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#67] +(67) CometHashAggregate +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(69) CometFilter +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(70) CometExchange +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 ASC NULLS FIRST] +(71) CometSort +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(73) CometFilter +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(74) CometProject +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(75) CometExchange +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(76) CometSort +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], LeftOuter +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(78) CometProject +Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +Right output [1]: [d_date_sk#65] +Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(81) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#66, web_site_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#83, 16, true, false, true) AS web_site_id#84] +(84) CometProject +Input [2]: [web_site_sk#66, web_site_id#67] +Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +Right output [2]: [web_site_sk#66, web_site_id#68] +Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(87) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] +Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [i_item_sk#69] +Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(90) CometProject +Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] +Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(92) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [p_promo_sk#70] +Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(93) CometProject +Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] +Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(94) CometHashAggregate +Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Keys [1]: [web_site_id#68] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(95) CometExchange +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] (96) CometHashAggregate -Input [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [web channel AS channel#95, concat(web_site, web_site_id#84) AS id#96, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#97, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#98, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#99] - -(100) Union - -(101) HashAggregate [codegen id : 4] -Input [5]: [channel#31, id#32, sales#33, returns#34, profit#35] -Keys [2]: [channel#31, id#32] -Functions [3]: [partial_sum(sales#33), partial_sum(returns#34), partial_sum(profit#35)] -Aggregate Attributes [6]: [sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Results [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(102) CometColumnarExchange -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(103) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(104) HashAggregate [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [5]: [channel#31, id#32, cast(sum(sales#33)#112 as decimal(37,2)) AS sales#115, cast(sum(returns#34)#113 as decimal(38,2)) AS returns#116, cast(sum(profit#35)#114 as decimal(38,2)) AS profit#117] - -(105) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(106) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(107) HashAggregate [codegen id : 10] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [4]: [channel#31, sum(sales#33)#112 AS sales#118, sum(returns#34)#113 AS returns#119, sum(profit#35)#114 AS profit#120] - -(108) HashAggregate [codegen id : 10] -Input [4]: [channel#31, sales#118, returns#119, profit#120] -Keys [1]: [channel#31] -Functions [3]: [partial_sum(sales#118), partial_sum(returns#119), partial_sum(profit#120)] -Aggregate Attributes [6]: [sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Results [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(109) CometColumnarExchange -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(110) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(111) HashAggregate [codegen id : 11] -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [1]: [channel#31] -Functions [3]: [sum(sales#118), sum(returns#119), sum(profit#120)] -Aggregate Attributes [3]: [sum(sales#118)#133, sum(returns#119)#134, sum(profit#120)#135] -Results [5]: [channel#31, null AS id#136, sum(sales#118)#133 AS sales#137, sum(returns#119)#134 AS returns#138, sum(profit#120)#135 AS profit#139] - -(112) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(113) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(114) HashAggregate [codegen id : 16] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [3]: [sum(sales#33)#112 AS sales#118, sum(returns#34)#113 AS returns#119, sum(profit#35)#114 AS profit#120] - -(115) HashAggregate [codegen id : 16] -Input [3]: [sales#118, returns#119, profit#120] +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Keys [1]: [web_site_id#68] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(97) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] +Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] + +(98) CometHashAggregate +Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Keys [2]: [channel#76, id#77] +Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] + +(99) CometExchange +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(100) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(101) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(102) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(103) CometHashAggregate +Input [4]: [channel#76, sales#97, returns#98, profit#99] +Keys [1]: [channel#76] +Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] + +(104) CometExchange +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(105) CometHashAggregate +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Keys [1]: [channel#76] +Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] + +(106) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(107) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(108) CometHashAggregate +Input [3]: [sales#97, returns#98, profit#99] Keys: [] -Functions [3]: [partial_sum(sales#118), partial_sum(returns#119), partial_sum(profit#120)] -Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Results [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] - -(116) CometColumnarExchange -Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] -(117) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +(109) CometExchange +Input [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(118) HashAggregate [codegen id : 17] -Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +(110) CometHashAggregate +Input [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] Keys: [] -Functions [3]: [sum(sales#118), sum(returns#119), sum(profit#120)] -Aggregate Attributes [3]: [sum(sales#118)#152, sum(returns#119)#153, sum(profit#120)#154] -Results [5]: [null AS channel#155, null AS id#156, sum(sales#118)#152 AS sales#157, sum(returns#119)#153 AS returns#158, sum(profit#120)#154 AS profit#159] +Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] -(119) Union +(111) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] +Child 1 Input [5]: [channel#76, id#115, sales#116, returns#117, profit#118] +Child 2 Input [5]: [channel#119, id#120, sales#121, returns#122, profit#123] -(120) HashAggregate [codegen id : 18] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(112) CometHashAggregate +Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] +Keys [5]: [channel#76, id#77, sales#112, returns#113, profit#114] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -(121) CometColumnarExchange -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: hashpartitioning(channel#31, id#32, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +(113) CometExchange +Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] +Arguments: hashpartitioning(channel#76, id#77, sales#112, returns#113, profit#114, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(122) CometHashAggregate -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(114) CometHashAggregate +Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] +Keys [5]: [channel#76, id#77, sales#112, returns#113, profit#114] Functions: [] -(123) CometTakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#115,returns#116,profit#117]), [channel#31, id#32, sales#115, returns#116, profit#117], 100, 0, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#115, returns#116, profit#117] +(115) CometTakeOrderedAndProject +Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#112,returns#113,profit#114]), [channel#76, id#77, sales#112, returns#113, profit#114], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#112, returns#113, profit#114] -(124) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(116) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) +BroadcastExchange (121) ++- * CometColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(117) 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,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(126) CometFilter +(118) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(127) CometProject +(119) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(128) CometColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(129) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/simplified.txt index f3a5ab15b6..d27ab827e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a.native_iceberg_compat/simplified.txt @@ -1,156 +1,128 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - 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] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [s_store_id] #3 + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + 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 [s_store_sk,s_store_id] #8 - 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] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - WholeStageCodegen (11) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #19 - WholeStageCodegen (10) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #20 - WholeStageCodegen (16) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + 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 [s_store_sk,s_store_id] #8 + 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] #9 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #10 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [cp_catalog_page_id] #11 + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #12 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [web_site_id] #15 + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #16 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #19 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange #20 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt index d6b731eb98..6898d458e7 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/explain.txt @@ -1,128 +1,120 @@ == Physical Plan == -* CometColumnarToRow (124) -+- CometTakeOrderedAndProject (123) - +- CometHashAggregate (122) - +- CometColumnarExchange (121) - +- * HashAggregate (120) - +- Union (119) - :- * HashAggregate (104) - : +- * CometColumnarToRow (103) - : +- CometColumnarExchange (102) - : +- * HashAggregate (101) - : +- Union (100) - : :- * HashAggregate (39) - : : +- * CometColumnarToRow (38) - : : +- CometExchange (37) - : : +- CometHashAggregate (36) - : : +- CometProject (35) - : : +- CometBroadcastHashJoin (34) - : : :- CometProject (29) - : : : +- CometBroadcastHashJoin (28) - : : : :- CometProject (23) - : : : : +- CometBroadcastHashJoin (22) - : : : : :- CometProject (17) - : : : : : +- CometBroadcastHashJoin (16) - : : : : : :- CometProject (11) - : : : : : : +- CometSortMergeJoin (10) - : : : : : : :- CometSort (4) - : : : : : : : +- CometExchange (3) - : : : : : : : +- CometFilter (2) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) - : : : : : : +- CometSort (9) - : : : : : : +- CometExchange (8) - : : : : : : +- CometProject (7) - : : : : : : +- CometFilter (6) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) - : : : : : +- CometBroadcastExchange (15) - : : : : : +- CometProject (14) - : : : : : +- CometFilter (13) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) - : : : : +- CometBroadcastExchange (21) - : : : : +- CometProject (20) - : : : : +- CometFilter (19) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) - : : : +- CometBroadcastExchange (27) - : : : +- CometProject (26) - : : : +- CometFilter (25) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) - : : +- CometBroadcastExchange (33) - : : +- CometProject (32) - : : +- CometFilter (31) - : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) - : :- * HashAggregate (69) - : : +- * CometColumnarToRow (68) - : : +- CometExchange (67) - : : +- CometHashAggregate (66) - : : +- CometProject (65) - : : +- CometBroadcastHashJoin (64) - : : :- CometProject (62) - : : : +- CometBroadcastHashJoin (61) - : : : :- CometProject (59) - : : : : +- CometBroadcastHashJoin (58) - : : : : :- CometProject (53) - : : : : : +- CometBroadcastHashJoin (52) - : : : : : :- CometProject (50) - : : : : : : +- CometSortMergeJoin (49) - : : : : : : :- CometSort (43) - : : : : : : : +- CometExchange (42) - : : : : : : : +- CometFilter (41) - : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (40) - : : : : : : +- CometSort (48) - : : : : : : +- CometExchange (47) - : : : : : : +- CometProject (46) - : : : : : : +- CometFilter (45) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (44) - : : : : : +- ReusedExchange (51) - : : : : +- CometBroadcastExchange (57) - : : : : +- CometProject (56) - : : : : +- CometFilter (55) - : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (54) - : : : +- ReusedExchange (60) - : : +- ReusedExchange (63) - : +- * HashAggregate (99) - : +- * CometColumnarToRow (98) - : +- CometExchange (97) - : +- CometHashAggregate (96) - : +- CometProject (95) - : +- CometBroadcastHashJoin (94) - : :- CometProject (92) - : : +- CometBroadcastHashJoin (91) - : : :- CometProject (89) - : : : +- CometBroadcastHashJoin (88) - : : : :- CometProject (83) - : : : : +- CometBroadcastHashJoin (82) - : : : : :- CometProject (80) - : : : : : +- CometSortMergeJoin (79) - : : : : : :- CometSort (73) - : : : : : : +- CometExchange (72) - : : : : : : +- CometFilter (71) - : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (70) - : : : : : +- CometSort (78) - : : : : : +- CometExchange (77) - : : : : : +- CometProject (76) - : : : : : +- CometFilter (75) - : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (74) - : : : : +- ReusedExchange (81) - : : : +- CometBroadcastExchange (87) - : : : +- CometProject (86) - : : : +- CometFilter (85) - : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (84) - : : +- ReusedExchange (90) - : +- ReusedExchange (93) - :- * HashAggregate (111) - : +- * CometColumnarToRow (110) - : +- CometColumnarExchange (109) - : +- * HashAggregate (108) - : +- * HashAggregate (107) - : +- * CometColumnarToRow (106) - : +- ReusedExchange (105) - +- * HashAggregate (118) - +- * CometColumnarToRow (117) - +- CometColumnarExchange (116) - +- * HashAggregate (115) - +- * HashAggregate (114) - +- * CometColumnarToRow (113) - +- ReusedExchange (112) +* CometColumnarToRow (116) ++- CometTakeOrderedAndProject (115) + +- CometHashAggregate (114) + +- CometExchange (113) + +- CometHashAggregate (112) + +- CometUnion (111) + :- CometHashAggregate (100) + : +- CometExchange (99) + : +- CometHashAggregate (98) + : +- CometUnion (97) + : :- CometHashAggregate (38) + : : +- CometExchange (37) + : : +- CometHashAggregate (36) + : : +- CometProject (35) + : : +- CometBroadcastHashJoin (34) + : : :- CometProject (29) + : : : +- CometBroadcastHashJoin (28) + : : : :- CometProject (23) + : : : : +- CometBroadcastHashJoin (22) + : : : : :- CometProject (17) + : : : : : +- CometBroadcastHashJoin (16) + : : : : : :- CometProject (11) + : : : : : : +- CometSortMergeJoin (10) + : : : : : : :- CometSort (4) + : : : : : : : +- CometExchange (3) + : : : : : : : +- CometFilter (2) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales (1) + : : : : : : +- CometSort (9) + : : : : : : +- CometExchange (8) + : : : : : : +- CometProject (7) + : : : : : : +- CometFilter (6) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns (5) + : : : : : +- CometBroadcastExchange (15) + : : : : : +- CometProject (14) + : : : : : +- CometFilter (13) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (12) + : : : : +- CometBroadcastExchange (21) + : : : : +- CometProject (20) + : : : : +- CometFilter (19) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.store (18) + : : : +- CometBroadcastExchange (27) + : : : +- CometProject (26) + : : : +- CometFilter (25) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.item (24) + : : +- CometBroadcastExchange (33) + : : +- CometProject (32) + : : +- CometFilter (31) + : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion (30) + : :- CometHashAggregate (67) + : : +- CometExchange (66) + : : +- CometHashAggregate (65) + : : +- CometProject (64) + : : +- CometBroadcastHashJoin (63) + : : :- CometProject (61) + : : : +- CometBroadcastHashJoin (60) + : : : :- CometProject (58) + : : : : +- CometBroadcastHashJoin (57) + : : : : :- CometProject (52) + : : : : : +- CometBroadcastHashJoin (51) + : : : : : :- CometProject (49) + : : : : : : +- CometSortMergeJoin (48) + : : : : : : :- CometSort (42) + : : : : : : : +- CometExchange (41) + : : : : : : : +- CometFilter (40) + : : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales (39) + : : : : : : +- CometSort (47) + : : : : : : +- CometExchange (46) + : : : : : : +- CometProject (45) + : : : : : : +- CometFilter (44) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns (43) + : : : : : +- ReusedExchange (50) + : : : : +- CometBroadcastExchange (56) + : : : : +- CometProject (55) + : : : : +- CometFilter (54) + : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page (53) + : : : +- ReusedExchange (59) + : : +- ReusedExchange (62) + : +- CometHashAggregate (96) + : +- CometExchange (95) + : +- CometHashAggregate (94) + : +- CometProject (93) + : +- CometBroadcastHashJoin (92) + : :- CometProject (90) + : : +- CometBroadcastHashJoin (89) + : : :- CometProject (87) + : : : +- CometBroadcastHashJoin (86) + : : : :- CometProject (81) + : : : : +- CometBroadcastHashJoin (80) + : : : : :- CometProject (78) + : : : : : +- CometSortMergeJoin (77) + : : : : : :- CometSort (71) + : : : : : : +- CometExchange (70) + : : : : : : +- CometFilter (69) + : : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales (68) + : : : : : +- CometSort (76) + : : : : : +- CometExchange (75) + : : : : : +- CometProject (74) + : : : : : +- CometFilter (73) + : : : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns (72) + : : : : +- ReusedExchange (79) + : : : +- CometBroadcastExchange (85) + : : : +- CometProject (84) + : : : +- CometFilter (83) + : : : +- CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site (82) + : : +- ReusedExchange (88) + : +- ReusedExchange (91) + :- CometHashAggregate (105) + : +- CometExchange (104) + : +- CometHashAggregate (103) + : +- CometHashAggregate (102) + : +- ReusedExchange (101) + +- CometHashAggregate (110) + +- CometExchange (109) + +- CometHashAggregate (108) + +- CometHashAggregate (107) + +- ReusedExchange (106) (1) CometScan [native_iceberg_compat] parquet spark_catalog.default.store_sales @@ -298,436 +290,394 @@ Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(co Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Arguments: hashpartitioning(s_store_id#18, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=3] -(38) CometColumnarToRow [codegen id : 1] -Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] - -(39) HashAggregate [codegen id : 1] +(38) CometHashAggregate Input [6]: [s_store_id#18, sum#23, sum#24, isEmpty#25, sum#26, isEmpty#27] Keys [1]: [s_store_id#18] Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#28, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30] -Results [5]: [store channel AS channel#31, concat(store, s_store_id#18) AS id#32, MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#28,17,2) AS sales#33, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#29 AS returns#34, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#30 AS profit#35] -(40) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales -Output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] +(39) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales +Output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(cs_sold_date_sk#42), dynamicpruningexpression(cs_sold_date_sk#42 IN dynamicpruning#43)] +PartitionFilters: [isnotnull(cs_sold_date_sk#34), dynamicpruningexpression(cs_sold_date_sk#34 IN dynamicpruning#35)] PushedFilters: [IsNotNull(cs_catalog_page_sk), IsNotNull(cs_item_sk), IsNotNull(cs_promo_sk)] ReadSchema: struct -(41) CometFilter -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Condition : ((isnotnull(cs_catalog_page_sk#36) AND isnotnull(cs_item_sk#37)) AND isnotnull(cs_promo_sk#38)) +(40) CometFilter +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Condition : ((isnotnull(cs_catalog_page_sk#28) AND isnotnull(cs_item_sk#29)) AND isnotnull(cs_promo_sk#30)) -(42) CometExchange -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: hashpartitioning(cs_item_sk#37, cs_order_number#39, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] +(41) CometExchange +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: hashpartitioning(cs_item_sk#29, cs_order_number#31, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=4] -(43) CometSort -Input [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42], [cs_item_sk#37 ASC NULLS FIRST, cs_order_number#39 ASC NULLS FIRST] +(42) CometSort +Input [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34], [cs_item_sk#29 ASC NULLS FIRST, cs_order_number#31 ASC NULLS FIRST] -(44) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns -Output [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] +(43) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns +Output [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_returns] PushedFilters: [IsNotNull(cr_item_sk), IsNotNull(cr_order_number)] ReadSchema: struct -(45) CometFilter -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Condition : (isnotnull(cr_item_sk#44) AND isnotnull(cr_order_number#45)) +(44) CometFilter +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Condition : (isnotnull(cr_item_sk#36) AND isnotnull(cr_order_number#37)) -(46) CometProject -Input [5]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47, cr_returned_date_sk#48] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] +(45) CometProject +Input [5]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39, cr_returned_date_sk#40] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] -(47) CometExchange -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: hashpartitioning(cr_item_sk#44, cr_order_number#45, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] +(46) CometExchange +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: hashpartitioning(cr_item_sk#36, cr_order_number#37, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=5] -(48) CometSort -Input [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47], [cr_item_sk#44 ASC NULLS FIRST, cr_order_number#45 ASC NULLS FIRST] +(47) CometSort +Input [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39], [cr_item_sk#36 ASC NULLS FIRST, cr_order_number#37 ASC NULLS FIRST] -(49) CometSortMergeJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42] -Right output [4]: [cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_item_sk#37, cs_order_number#39], [cr_item_sk#44, cr_order_number#45], LeftOuter +(48) CometSortMergeJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34] +Right output [4]: [cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_item_sk#29, cs_order_number#31], [cr_item_sk#36, cr_order_number#37], LeftOuter -(50) CometProject -Input [11]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_order_number#39, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_item_sk#44, cr_order_number#45, cr_return_amount#46, cr_net_loss#47] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] +(49) CometProject +Input [11]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_order_number#31, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_item_sk#36, cr_order_number#37, cr_return_amount#38, cr_net_loss#39] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] -(51) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#49] +(50) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#41] -(52) CometBroadcastHashJoin -Left output [8]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47] -Right output [1]: [d_date_sk#49] -Arguments: [cs_sold_date_sk#42], [d_date_sk#49], Inner, BuildRight +(51) CometBroadcastHashJoin +Left output [8]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39] +Right output [1]: [d_date_sk#41] +Arguments: [cs_sold_date_sk#34], [d_date_sk#41], Inner, BuildRight -(53) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cs_sold_date_sk#42, cr_return_amount#46, cr_net_loss#47, d_date_sk#49] -Arguments: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47], [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] +(52) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cs_sold_date_sk#34, cr_return_amount#38, cr_net_loss#39, d_date_sk#41] +Arguments: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39], [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] -(54) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page -Output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] +(53) CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page +Output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] Batched: true Location [not included in comparison]/{warehouse_dir}/catalog_page] PushedFilters: [IsNotNull(cp_catalog_page_sk)] ReadSchema: struct -(55) CometFilter -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Condition : isnotnull(cp_catalog_page_sk#50) - -(56) CometProject -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#51] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52], [cp_catalog_page_sk#50, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#51, 16, true, false, true) AS cp_catalog_page_id#52] +(54) CometFilter +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Condition : isnotnull(cp_catalog_page_sk#42) -(57) CometBroadcastExchange -Input [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cp_catalog_page_sk#50, cp_catalog_page_id#52] +(55) CometProject +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#43] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44], [cp_catalog_page_sk#42, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, cp_catalog_page_id#43, 16, true, false, true) AS cp_catalog_page_id#44] -(58) CometBroadcastHashJoin -Left output [7]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47] -Right output [2]: [cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_catalog_page_sk#36], [cp_catalog_page_sk#50], Inner, BuildRight +(56) CometBroadcastExchange +Input [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cp_catalog_page_sk#42, cp_catalog_page_id#44] -(59) CometProject -Input [9]: [cs_catalog_page_sk#36, cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_sk#50, cp_catalog_page_id#52] -Arguments: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(57) CometBroadcastHashJoin +Left output [7]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39] +Right output [2]: [cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_catalog_page_sk#28], [cp_catalog_page_sk#42], Inner, BuildRight -(60) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#53] +(58) CometProject +Input [9]: [cs_catalog_page_sk#28, cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_sk#42, cp_catalog_page_id#44] +Arguments: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(61) CometBroadcastHashJoin -Left output [7]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [i_item_sk#53] -Arguments: [cs_item_sk#37], [i_item_sk#53], Inner, BuildRight +(59) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#45] -(62) CometProject -Input [8]: [cs_item_sk#37, cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, i_item_sk#53] -Arguments: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(60) CometBroadcastHashJoin +Left output [7]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [i_item_sk#45] +Arguments: [cs_item_sk#29], [i_item_sk#45], Inner, BuildRight -(63) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#54] +(61) CometProject +Input [8]: [cs_item_sk#29, cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, i_item_sk#45] +Arguments: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(64) CometBroadcastHashJoin -Left output [6]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Right output [1]: [p_promo_sk#54] -Arguments: [cs_promo_sk#38], [p_promo_sk#54], Inner, BuildRight +(62) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#46] -(65) CometProject -Input [7]: [cs_promo_sk#38, cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52, p_promo_sk#54] -Arguments: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52], [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] +(63) CometBroadcastHashJoin +Left output [6]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Right output [1]: [p_promo_sk#46] +Arguments: [cs_promo_sk#30], [p_promo_sk#46], Inner, BuildRight -(66) CometHashAggregate -Input [5]: [cs_ext_sales_price#40, cs_net_profit#41, cr_return_amount#46, cr_net_loss#47, cp_catalog_page_id#52] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#40)), partial_sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] +(64) CometProject +Input [7]: [cs_promo_sk#30, cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44, p_promo_sk#46] +Arguments: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44], [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] -(67) CometExchange -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Arguments: hashpartitioning(cp_catalog_page_id#52, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] +(65) CometHashAggregate +Input [5]: [cs_ext_sales_price#32, cs_net_profit#33, cr_return_amount#38, cr_net_loss#39, cp_catalog_page_id#44] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#32)), partial_sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(68) CometColumnarToRow [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] +(66) CometExchange +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Arguments: hashpartitioning(cp_catalog_page_id#44, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=6] -(69) HashAggregate [codegen id : 2] -Input [6]: [cp_catalog_page_id#52, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#59] -Keys [1]: [cp_catalog_page_id#52] -Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#40)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#40))#60, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62] -Results [5]: [catalog channel AS channel#63, concat(catalog_page, cp_catalog_page_id#52) AS id#64, MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#40))#60,17,2) AS sales#65, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#61 AS returns#66, sum((cs_net_profit#41 - coalesce(cast(cr_net_loss#47 as decimal(12,2)), 0.00)))#62 AS profit#67] +(67) CometHashAggregate +Input [6]: [cp_catalog_page_id#44, sum#47, sum#48, isEmpty#49, sum#50, isEmpty#51] +Keys [1]: [cp_catalog_page_id#44] +Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#32)), sum(coalesce(cast(cr_return_amount#38 as decimal(12,2)), 0.00)), sum((cs_net_profit#33 - coalesce(cast(cr_net_loss#39 as decimal(12,2)), 0.00)))] -(70) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales -Output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] +(68) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales +Output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] Batched: true Location: InMemoryFileIndex [] -PartitionFilters: [isnotnull(ws_sold_date_sk#74), dynamicpruningexpression(ws_sold_date_sk#74 IN dynamicpruning#75)] +PartitionFilters: [isnotnull(ws_sold_date_sk#58), dynamicpruningexpression(ws_sold_date_sk#58 IN dynamicpruning#59)] PushedFilters: [IsNotNull(ws_web_site_sk), IsNotNull(ws_item_sk), IsNotNull(ws_promo_sk)] ReadSchema: struct -(71) CometFilter -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Condition : ((isnotnull(ws_web_site_sk#69) AND isnotnull(ws_item_sk#68)) AND isnotnull(ws_promo_sk#70)) +(69) CometFilter +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Condition : ((isnotnull(ws_web_site_sk#53) AND isnotnull(ws_item_sk#52)) AND isnotnull(ws_promo_sk#54)) -(72) CometExchange -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: hashpartitioning(ws_item_sk#68, ws_order_number#71, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] +(70) CometExchange +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: hashpartitioning(ws_item_sk#52, ws_order_number#55, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=7] -(73) CometSort -Input [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74], [ws_item_sk#68 ASC NULLS FIRST, ws_order_number#71 ASC NULLS FIRST] +(71) CometSort +Input [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58], [ws_item_sk#52 ASC NULLS FIRST, ws_order_number#55 ASC NULLS FIRST] -(74) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns -Output [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] +(72) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns +Output [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] Batched: true Location [not included in comparison]/{warehouse_dir}/web_returns] PushedFilters: [IsNotNull(wr_item_sk), IsNotNull(wr_order_number)] ReadSchema: struct -(75) CometFilter -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Condition : (isnotnull(wr_item_sk#76) AND isnotnull(wr_order_number#77)) +(73) CometFilter +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Condition : (isnotnull(wr_item_sk#60) AND isnotnull(wr_order_number#61)) -(76) CometProject -Input [5]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79, wr_returned_date_sk#80] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] +(74) CometProject +Input [5]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63, wr_returned_date_sk#64] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] -(77) CometExchange -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: hashpartitioning(wr_item_sk#76, wr_order_number#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] +(75) CometExchange +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: hashpartitioning(wr_item_sk#60, wr_order_number#61, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=8] -(78) CometSort -Input [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79], [wr_item_sk#76 ASC NULLS FIRST, wr_order_number#77 ASC NULLS FIRST] +(76) CometSort +Input [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63], [wr_item_sk#60 ASC NULLS FIRST, wr_order_number#61 ASC NULLS FIRST] -(79) CometSortMergeJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74] -Right output [4]: [wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_order_number#71], [wr_item_sk#76, wr_order_number#77], LeftOuter +(77) CometSortMergeJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58] +Right output [4]: [wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_order_number#55], [wr_item_sk#60, wr_order_number#61], LeftOuter -(80) CometProject -Input [11]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_order_number#71, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_item_sk#76, wr_order_number#77, wr_return_amt#78, wr_net_loss#79] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] +(78) CometProject +Input [11]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_order_number#55, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_item_sk#60, wr_order_number#61, wr_return_amt#62, wr_net_loss#63] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] -(81) ReusedExchange [Reuses operator id: 15] -Output [1]: [d_date_sk#81] +(79) ReusedExchange [Reuses operator id: 15] +Output [1]: [d_date_sk#65] -(82) CometBroadcastHashJoin -Left output [8]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79] -Right output [1]: [d_date_sk#81] -Arguments: [ws_sold_date_sk#74], [d_date_sk#81], Inner, BuildRight +(80) CometBroadcastHashJoin +Left output [8]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63] +Right output [1]: [d_date_sk#65] +Arguments: [ws_sold_date_sk#58], [d_date_sk#65], Inner, BuildRight -(83) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, ws_sold_date_sk#74, wr_return_amt#78, wr_net_loss#79, d_date_sk#81] -Arguments: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79], [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] +(81) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, ws_sold_date_sk#58, wr_return_amt#62, wr_net_loss#63, d_date_sk#65] +Arguments: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63], [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] -(84) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site -Output [2]: [web_site_sk#82, web_site_id#83] +(82) CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site +Output [2]: [web_site_sk#66, web_site_id#67] Batched: true Location [not included in comparison]/{warehouse_dir}/web_site] PushedFilters: [IsNotNull(web_site_sk)] ReadSchema: struct -(85) CometFilter -Input [2]: [web_site_sk#82, web_site_id#83] -Condition : isnotnull(web_site_sk#82) +(83) CometFilter +Input [2]: [web_site_sk#66, web_site_id#67] +Condition : isnotnull(web_site_sk#66) -(86) CometProject -Input [2]: [web_site_sk#82, web_site_id#83] -Arguments: [web_site_sk#82, web_site_id#84], [web_site_sk#82, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#83, 16, true, false, true) AS web_site_id#84] +(84) CometProject +Input [2]: [web_site_sk#66, web_site_id#67] +Arguments: [web_site_sk#66, web_site_id#68], [web_site_sk#66, staticinvoke(class org.apache.spark.sql.catalyst.util.CharVarcharCodegenUtils, StringType, readSidePadding, web_site_id#67, 16, true, false, true) AS web_site_id#68] -(87) CometBroadcastExchange -Input [2]: [web_site_sk#82, web_site_id#84] -Arguments: [web_site_sk#82, web_site_id#84] +(85) CometBroadcastExchange +Input [2]: [web_site_sk#66, web_site_id#68] +Arguments: [web_site_sk#66, web_site_id#68] -(88) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79] -Right output [2]: [web_site_sk#82, web_site_id#84] -Arguments: [ws_web_site_sk#69], [web_site_sk#82], Inner, BuildRight +(86) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63] +Right output [2]: [web_site_sk#66, web_site_id#68] +Arguments: [ws_web_site_sk#53], [web_site_sk#66], Inner, BuildRight -(89) CometProject -Input [9]: [ws_item_sk#68, ws_web_site_sk#69, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_sk#82, web_site_id#84] -Arguments: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(87) CometProject +Input [9]: [ws_item_sk#52, ws_web_site_sk#53, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_sk#66, web_site_id#68] +Arguments: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(90) ReusedExchange [Reuses operator id: 27] -Output [1]: [i_item_sk#85] +(88) ReusedExchange [Reuses operator id: 27] +Output [1]: [i_item_sk#69] -(91) CometBroadcastHashJoin -Left output [7]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [i_item_sk#85] -Arguments: [ws_item_sk#68], [i_item_sk#85], Inner, BuildRight +(89) CometBroadcastHashJoin +Left output [7]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [i_item_sk#69] +Arguments: [ws_item_sk#52], [i_item_sk#69], Inner, BuildRight -(92) CometProject -Input [8]: [ws_item_sk#68, ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, i_item_sk#85] -Arguments: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(90) CometProject +Input [8]: [ws_item_sk#52, ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, i_item_sk#69] +Arguments: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] -(93) ReusedExchange [Reuses operator id: 33] -Output [1]: [p_promo_sk#86] +(91) ReusedExchange [Reuses operator id: 33] +Output [1]: [p_promo_sk#70] -(94) CometBroadcastHashJoin -Left output [6]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Right output [1]: [p_promo_sk#86] -Arguments: [ws_promo_sk#70], [p_promo_sk#86], Inner, BuildRight +(92) CometBroadcastHashJoin +Left output [6]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Right output [1]: [p_promo_sk#70] +Arguments: [ws_promo_sk#54], [p_promo_sk#70], Inner, BuildRight -(95) CometProject -Input [7]: [ws_promo_sk#70, ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84, p_promo_sk#86] -Arguments: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84], [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] +(93) CometProject +Input [7]: [ws_promo_sk#54, ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68, p_promo_sk#70] +Arguments: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68], [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] + +(94) CometHashAggregate +Input [5]: [ws_ext_sales_price#56, ws_net_profit#57, wr_return_amt#62, wr_net_loss#63, web_site_id#68] +Keys [1]: [web_site_id#68] +Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#56)), partial_sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(95) CometExchange +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Arguments: hashpartitioning(web_site_id#68, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] (96) CometHashAggregate -Input [5]: [ws_ext_sales_price#72, ws_net_profit#73, wr_return_amt#78, wr_net_loss#79, web_site_id#84] -Keys [1]: [web_site_id#84] -Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#72)), partial_sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] - -(97) CometExchange -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Arguments: hashpartitioning(web_site_id#84, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=9] - -(98) CometColumnarToRow [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] - -(99) HashAggregate [codegen id : 3] -Input [6]: [web_site_id#84, sum#87, sum#88, isEmpty#89, sum#90, isEmpty#91] -Keys [1]: [web_site_id#84] -Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#72)), sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00)), sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))] -Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#72))#92, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94] -Results [5]: [web channel AS channel#95, concat(web_site, web_site_id#84) AS id#96, MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#72))#92,17,2) AS sales#97, sum(coalesce(cast(wr_return_amt#78 as decimal(12,2)), 0.00))#93 AS returns#98, sum((ws_net_profit#73 - coalesce(cast(wr_net_loss#79 as decimal(12,2)), 0.00)))#94 AS profit#99] - -(100) Union - -(101) HashAggregate [codegen id : 4] -Input [5]: [channel#31, id#32, sales#33, returns#34, profit#35] -Keys [2]: [channel#31, id#32] -Functions [3]: [partial_sum(sales#33), partial_sum(returns#34), partial_sum(profit#35)] -Aggregate Attributes [6]: [sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] -Results [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(102) CometColumnarExchange -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Arguments: hashpartitioning(channel#31, id#32, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=10] - -(103) CometColumnarToRow [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(104) HashAggregate [codegen id : 5] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [5]: [channel#31, id#32, cast(sum(sales#33)#112 as decimal(37,2)) AS sales#115, cast(sum(returns#34)#113 as decimal(38,2)) AS returns#116, cast(sum(profit#35)#114 as decimal(38,2)) AS profit#117] - -(105) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(106) CometColumnarToRow [codegen id : 10] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(107) HashAggregate [codegen id : 10] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [4]: [channel#31, sum(sales#33)#112 AS sales#118, sum(returns#34)#113 AS returns#119, sum(profit#35)#114 AS profit#120] - -(108) HashAggregate [codegen id : 10] -Input [4]: [channel#31, sales#118, returns#119, profit#120] -Keys [1]: [channel#31] -Functions [3]: [partial_sum(sales#118), partial_sum(returns#119), partial_sum(profit#120)] -Aggregate Attributes [6]: [sum#121, isEmpty#122, sum#123, isEmpty#124, sum#125, isEmpty#126] -Results [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(109) CometColumnarExchange -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Arguments: hashpartitioning(channel#31, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=11] - -(110) CometColumnarToRow [codegen id : 11] -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] - -(111) HashAggregate [codegen id : 11] -Input [7]: [channel#31, sum#127, isEmpty#128, sum#129, isEmpty#130, sum#131, isEmpty#132] -Keys [1]: [channel#31] -Functions [3]: [sum(sales#118), sum(returns#119), sum(profit#120)] -Aggregate Attributes [3]: [sum(sales#118)#133, sum(returns#119)#134, sum(profit#120)#135] -Results [5]: [channel#31, null AS id#136, sum(sales#118)#133 AS sales#137, sum(returns#119)#134 AS returns#138, sum(profit#120)#135 AS profit#139] - -(112) ReusedExchange [Reuses operator id: 102] -Output [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(113) CometColumnarToRow [codegen id : 16] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] - -(114) HashAggregate [codegen id : 16] -Input [8]: [channel#31, id#32, sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] -Keys [2]: [channel#31, id#32] -Functions [3]: [sum(sales#33), sum(returns#34), sum(profit#35)] -Aggregate Attributes [3]: [sum(sales#33)#112, sum(returns#34)#113, sum(profit#35)#114] -Results [3]: [sum(sales#33)#112 AS sales#118, sum(returns#34)#113 AS returns#119, sum(profit#35)#114 AS profit#120] - -(115) HashAggregate [codegen id : 16] -Input [3]: [sales#118, returns#119, profit#120] +Input [6]: [web_site_id#68, sum#71, sum#72, isEmpty#73, sum#74, isEmpty#75] +Keys [1]: [web_site_id#68] +Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#56)), sum(coalesce(cast(wr_return_amt#62 as decimal(12,2)), 0.00)), sum((ws_net_profit#57 - coalesce(cast(wr_net_loss#63 as decimal(12,2)), 0.00)))] + +(97) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Child 1 Input [5]: [channel#81, id#82, sales#83, returns#84, profit#85] +Child 2 Input [5]: [channel#86, id#87, sales#88, returns#89, profit#90] + +(98) CometHashAggregate +Input [5]: [channel#76, id#77, sales#78, returns#79, profit#80] +Keys [2]: [channel#76, id#77] +Functions [3]: [partial_sum(sales#78), partial_sum(returns#79), partial_sum(profit#80)] + +(99) CometExchange +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Arguments: hashpartitioning(channel#76, id#77, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=10] + +(100) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(101) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(102) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(103) CometHashAggregate +Input [4]: [channel#76, sales#97, returns#98, profit#99] +Keys [1]: [channel#76] +Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] + +(104) CometExchange +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Arguments: hashpartitioning(channel#76, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=11] + +(105) CometHashAggregate +Input [7]: [channel#76, sum#100, isEmpty#101, sum#102, isEmpty#103, sum#104, isEmpty#105] +Keys [1]: [channel#76] +Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] + +(106) ReusedExchange [Reuses operator id: 99] +Output [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] + +(107) CometHashAggregate +Input [8]: [channel#76, id#77, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96] +Keys [2]: [channel#76, id#77] +Functions [3]: [sum(sales#78), sum(returns#79), sum(profit#80)] + +(108) CometHashAggregate +Input [3]: [sales#97, returns#98, profit#99] Keys: [] -Functions [3]: [partial_sum(sales#118), partial_sum(returns#119), partial_sum(profit#120)] -Aggregate Attributes [6]: [sum#140, isEmpty#141, sum#142, isEmpty#143, sum#144, isEmpty#145] -Results [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] - -(116) CometColumnarExchange -Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] -Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=12] +Functions [3]: [partial_sum(sales#97), partial_sum(returns#98), partial_sum(profit#99)] -(117) CometColumnarToRow [codegen id : 17] -Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +(109) CometExchange +Input [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] +Arguments: SinglePartition, ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=12] -(118) HashAggregate [codegen id : 17] -Input [6]: [sum#146, isEmpty#147, sum#148, isEmpty#149, sum#150, isEmpty#151] +(110) CometHashAggregate +Input [6]: [sum#106, isEmpty#107, sum#108, isEmpty#109, sum#110, isEmpty#111] Keys: [] -Functions [3]: [sum(sales#118), sum(returns#119), sum(profit#120)] -Aggregate Attributes [3]: [sum(sales#118)#152, sum(returns#119)#153, sum(profit#120)#154] -Results [5]: [null AS channel#155, null AS id#156, sum(sales#118)#152 AS sales#157, sum(returns#119)#153 AS returns#158, sum(profit#120)#154 AS profit#159] +Functions [3]: [sum(sales#97), sum(returns#98), sum(profit#99)] -(119) Union +(111) CometUnion +Child 0 Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] +Child 1 Input [5]: [channel#76, id#115, sales#116, returns#117, profit#118] +Child 2 Input [5]: [channel#119, id#120, sales#121, returns#122, profit#123] -(120) HashAggregate [codegen id : 18] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(112) CometHashAggregate +Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] +Keys [5]: [channel#76, id#77, sales#112, returns#113, profit#114] Functions: [] -Aggregate Attributes: [] -Results [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -(121) CometColumnarExchange -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: hashpartitioning(channel#31, id#32, sales#115, returns#116, profit#117, 5), ENSURE_REQUIREMENTS, CometColumnarShuffle, [plan_id=13] +(113) CometExchange +Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] +Arguments: hashpartitioning(channel#76, id#77, sales#112, returns#113, profit#114, 5), ENSURE_REQUIREMENTS, CometNativeShuffle, [plan_id=13] -(122) CometHashAggregate -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Keys [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(114) CometHashAggregate +Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] +Keys [5]: [channel#76, id#77, sales#112, returns#113, profit#114] Functions: [] -(123) CometTakeOrderedAndProject -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] -Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#31 ASC NULLS FIRST,id#32 ASC NULLS FIRST], output=[channel#31,id#32,sales#115,returns#116,profit#117]), [channel#31, id#32, sales#115, returns#116, profit#117], 100, 0, [channel#31 ASC NULLS FIRST, id#32 ASC NULLS FIRST], [channel#31, id#32, sales#115, returns#116, profit#117] +(115) CometTakeOrderedAndProject +Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] +Arguments: TakeOrderedAndProject(limit=100, orderBy=[channel#76 ASC NULLS FIRST,id#77 ASC NULLS FIRST], output=[channel#76,id#77,sales#112,returns#113,profit#114]), [channel#76, id#77, sales#112, returns#113, profit#114], 100, 0, [channel#76 ASC NULLS FIRST, id#77 ASC NULLS FIRST], [channel#76, id#77, sales#112, returns#113, profit#114] -(124) CometColumnarToRow [codegen id : 19] -Input [5]: [channel#31, id#32, sales#115, returns#116, profit#117] +(116) CometColumnarToRow [codegen id : 1] +Input [5]: [channel#76, id#77, sales#112, returns#113, profit#114] ===== Subqueries ===== Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#7 IN dynamicpruning#8 -BroadcastExchange (129) -+- * CometColumnarToRow (128) - +- CometProject (127) - +- CometFilter (126) - +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (125) +BroadcastExchange (121) ++- * CometColumnarToRow (120) + +- CometProject (119) + +- CometFilter (118) + +- CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim (117) -(125) CometScan [native_iceberg_compat] parquet spark_catalog.default.date_dim +(117) 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,1998-08-04), LessThanOrEqual(d_date,1998-09-03), IsNotNull(d_date_sk)] ReadSchema: struct -(126) CometFilter +(118) CometFilter Input [2]: [d_date_sk#14, d_date#15] Condition : (((isnotnull(d_date#15) AND (d_date#15 >= 1998-08-04)) AND (d_date#15 <= 1998-09-03)) AND isnotnull(d_date_sk#14)) -(127) CometProject +(119) CometProject Input [2]: [d_date_sk#14, d_date#15] Arguments: [d_date_sk#14], [d_date_sk#14] -(128) CometColumnarToRow [codegen id : 1] +(120) CometColumnarToRow [codegen id : 1] Input [1]: [d_date_sk#14] -(129) BroadcastExchange +(121) BroadcastExchange Input [1]: [d_date_sk#14] Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14] -Subquery:2 Hosting operator id = 40 Hosting Expression = cs_sold_date_sk#42 IN dynamicpruning#8 +Subquery:2 Hosting operator id = 39 Hosting Expression = cs_sold_date_sk#34 IN dynamicpruning#8 -Subquery:3 Hosting operator id = 70 Hosting Expression = ws_sold_date_sk#74 IN dynamicpruning#8 +Subquery:3 Hosting operator id = 68 Hosting Expression = ws_sold_date_sk#58 IN dynamicpruning#8 diff --git a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt index f3a5ab15b6..d27ab827e1 100644 --- a/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt +++ b/spark/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q80a/simplified.txt @@ -1,156 +1,128 @@ -WholeStageCodegen (19) +WholeStageCodegen (1) CometColumnarToRow InputAdapter CometTakeOrderedAndProject [channel,id,sales,returns,profit] CometHashAggregate [channel,id,sales,returns,profit] - CometColumnarExchange [channel,id,sales,returns,profit] #1 - WholeStageCodegen (18) - HashAggregate [channel,id,sales,returns,profit] - InputAdapter - Union - WholeStageCodegen (5) - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel,id] #2 - WholeStageCodegen (4) - HashAggregate [channel,id,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - InputAdapter - Union - WholeStageCodegen (1) - HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [s_store_id] #3 - CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] - CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] - CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] - CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] - CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] - CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - CometExchange [ss_item_sk,ss_ticket_number] #4 - CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] - SubqueryBroadcast [d_date_sk] #1 - BroadcastExchange #5 - 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] - CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometExchange [sr_item_sk,sr_ticket_number] #6 - CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] - CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] - CometBroadcastExchange [d_date_sk] #7 + CometExchange [channel,id,sales,returns,profit] #1 + CometHashAggregate [channel,id,sales,returns,profit] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel,id] #2 + CometHashAggregate [sales,returns,profit] [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometUnion [channel,id,sales,returns,profit] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,s_store_id,sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [s_store_id] #3 + CometHashAggregate [ss_ext_sales_price,sr_return_amt,ss_net_profit,sr_net_loss] [s_store_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,p_promo_sk] + CometProject [ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id,i_item_sk] + CometProject [ss_item_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_id] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss,s_store_sk,s_store_id] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,sr_return_amt,sr_net_loss] + CometBroadcastHashJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss,d_date_sk] + CometProject [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_return_amt,sr_net_loss] + CometSortMergeJoin [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk,sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometSort [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + CometExchange [ss_item_sk,ss_ticket_number] #4 + CometFilter [ss_item_sk,ss_store_sk,ss_promo_sk,ss_ticket_number,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_promo_sk,ss_ticket_number,ss_ext_sales_price,ss_net_profit,ss_sold_date_sk] + SubqueryBroadcast [d_date_sk] #1 + BroadcastExchange #5 + 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 [s_store_sk,s_store_id] #8 - 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] #9 - CometProject [i_item_sk] - CometFilter [i_item_sk,i_current_price] - CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] - CometBroadcastExchange [p_promo_sk] #10 - CometProject [p_promo_sk] - CometFilter [p_promo_sk,p_channel_tv] - CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] - WholeStageCodegen (2) - HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [cp_catalog_page_id] #11 - CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] - CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] - CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] - CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] - CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] - CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometExchange [cs_item_sk,cs_order_number] #12 - CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometExchange [cr_item_sk,cr_order_number] #13 - CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] - CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 - CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] - CometFilter [cp_catalog_page_sk,cp_catalog_page_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - WholeStageCodegen (3) - HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),channel,id,sales,returns,profit,sum,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometExchange [web_site_id] #15 - CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] - CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] - CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] - CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] - CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] - CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] - CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometExchange [ws_item_sk,ws_order_number] #16 - CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] - ReusedSubquery [d_date_sk] #1 - CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometExchange [wr_item_sk,wr_order_number] #17 - CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] - CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] - ReusedExchange [d_date_sk] #7 - CometBroadcastExchange [web_site_sk,web_site_id] #18 - CometProject [web_site_id] [web_site_sk,web_site_id] - CometFilter [web_site_sk,web_site_id] - CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] - ReusedExchange [i_item_sk] #9 - ReusedExchange [p_promo_sk] #10 - WholeStageCodegen (11) - HashAggregate [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange [channel] #19 - WholeStageCodegen (10) - HashAggregate [channel,sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 - WholeStageCodegen (17) - HashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),channel,id,sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - CometColumnarExchange #20 - WholeStageCodegen (16) - HashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] - HashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(sales),sum(returns),sum(profit),sales,returns,profit,sum,isEmpty,sum,isEmpty,sum,isEmpty] - CometColumnarToRow - InputAdapter - ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometSort [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometExchange [sr_item_sk,sr_ticket_number] #6 + CometProject [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss] + CometFilter [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.store_returns [sr_item_sk,sr_ticket_number,sr_return_amt,sr_net_loss,sr_returned_date_sk] + CometBroadcastExchange [d_date_sk] #7 + 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 [s_store_sk,s_store_id] #8 + 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] #9 + CometProject [i_item_sk] + CometFilter [i_item_sk,i_current_price] + CometScan [native_iceberg_compat] parquet spark_catalog.default.item [i_item_sk,i_current_price] + CometBroadcastExchange [p_promo_sk] #10 + CometProject [p_promo_sk] + CometFilter [p_promo_sk,p_channel_tv] + CometScan [native_iceberg_compat] parquet spark_catalog.default.promotion [p_promo_sk,p_channel_tv] + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,cp_catalog_page_id,sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [cp_catalog_page_id] #11 + CometHashAggregate [cs_ext_sales_price,cr_return_amount,cs_net_profit,cr_net_loss] [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,p_promo_sk] + CometProject [cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id,i_item_sk] + CometProject [cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_id] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss,cp_catalog_page_sk,cp_catalog_page_id] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cr_return_amount,cr_net_loss] + CometBroadcastHashJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss,d_date_sk] + CometProject [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_return_amount,cr_net_loss] + CometSortMergeJoin [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk,cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometSort [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometExchange [cs_item_sk,cs_order_number] #12 + CometFilter [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_sales [cs_catalog_page_sk,cs_item_sk,cs_promo_sk,cs_order_number,cs_ext_sales_price,cs_net_profit,cs_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometExchange [cr_item_sk,cr_order_number] #13 + CometProject [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss] + CometFilter [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_returns [cr_item_sk,cr_order_number,cr_return_amount,cr_net_loss,cr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [cp_catalog_page_sk,cp_catalog_page_id] #14 + CometProject [cp_catalog_page_id] [cp_catalog_page_sk,cp_catalog_page_id] + CometFilter [cp_catalog_page_sk,cp_catalog_page_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,web_site_id,sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00)))] + CometExchange [web_site_id] #15 + CometHashAggregate [ws_ext_sales_price,wr_return_amt,ws_net_profit,wr_net_loss] [web_site_id,sum,sum,isEmpty,sum,isEmpty] + CometProject [ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,p_promo_sk] + CometProject [ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id,i_item_sk] + CometProject [ws_item_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_id] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss,web_site_sk,web_site_id] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,wr_return_amt,wr_net_loss] + CometBroadcastHashJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss,d_date_sk] + CometProject [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_return_amt,wr_net_loss] + CometSortMergeJoin [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk,wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometSort [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometExchange [ws_item_sk,ws_order_number] #16 + CometFilter [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_sales [ws_item_sk,ws_web_site_sk,ws_promo_sk,ws_order_number,ws_ext_sales_price,ws_net_profit,ws_sold_date_sk] + ReusedSubquery [d_date_sk] #1 + CometSort [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometExchange [wr_item_sk,wr_order_number] #17 + CometProject [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss] + CometFilter [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_returns [wr_item_sk,wr_order_number,wr_return_amt,wr_net_loss,wr_returned_date_sk] + ReusedExchange [d_date_sk] #7 + CometBroadcastExchange [web_site_sk,web_site_id] #18 + CometProject [web_site_id] [web_site_sk,web_site_id] + CometFilter [web_site_sk,web_site_id] + CometScan [native_iceberg_compat] parquet spark_catalog.default.web_site [web_site_sk,web_site_id] + ReusedExchange [i_item_sk] #9 + ReusedExchange [p_promo_sk] #10 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange [channel] #19 + CometHashAggregate [sales,returns,profit] [channel,sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 + CometHashAggregate [sum,isEmpty,sum,isEmpty,sum,isEmpty] [channel,id,sales,returns,profit,sum(sales),sum(returns),sum(profit)] + CometExchange #20 + CometHashAggregate [sales,returns,profit] [sum,isEmpty,sum,isEmpty,sum,isEmpty] + CometHashAggregate [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sales,returns,profit,sum(sales),sum(returns),sum(profit)] + ReusedExchange [channel,id,sum,isEmpty,sum,isEmpty,sum,isEmpty] #2 From 164dbfe550e2e99a7b4b64144309e09763f62384 Mon Sep 17 00:00:00 2001 From: comphead Date: Wed, 29 Oct 2025 17:56:22 -0700 Subject: [PATCH 8/8] feat: support `concat` for strings --- .../scala/org/apache/comet/fuzz/Meta.scala | 8 --- .../apache/comet/serde/QueryPlanSerde.scala | 2 +- .../org/apache/comet/serde/strings.scala | 14 +++++- .../apache/comet/CometExpressionSuite.scala | 49 +++++++++++++------ 4 files changed, 48 insertions(+), 25 deletions(-) diff --git a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala index 9948c428b2..70bc5bfe39 100644 --- a/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala +++ b/fuzz-testing/src/main/scala/org/apache/comet/fuzz/Meta.scala @@ -126,21 +126,13 @@ object Meta { SparkTypeOneOf( Seq( SparkStringType, - SparkNumericType, - SparkBinaryType, SparkArrayType( SparkTypeOneOf(Seq(SparkStringType, SparkNumericType, SparkBinaryType))))), SparkTypeOneOf( Seq( SparkStringType, - SparkNumericType, - SparkBinaryType, SparkArrayType( SparkTypeOneOf(Seq(SparkStringType, SparkNumericType, SparkBinaryType))))))), - createFunctionWithInputTypes( - "concat", - Seq(SparkStringType, SparkStringType) - ), // TODO: variadic createFunctionWithInputTypes("concat_ws", Seq(SparkStringType, SparkStringType)), createFunctionWithInputTypes("contains", Seq(SparkStringType, SparkStringType)), createFunctionWithInputTypes("ends_with", Seq(SparkStringType, SparkStringType)), 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 655f341fc2..570c07cb09 100644 --- a/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala +++ b/spark/src/main/scala/org/apache/comet/serde/QueryPlanSerde.scala @@ -164,7 +164,7 @@ object QueryPlanSerde extends Logging with CometExprShim { classOf[BitLength] -> CometScalarFunction("bit_length"), classOf[Chr] -> CometScalarFunction("char"), classOf[ConcatWs] -> CometScalarFunction("concat_ws"), - classOf[Concat] -> CometScalarFunction("concat"), + classOf[Concat] -> CometConcat, classOf[Contains] -> CometScalarFunction("contains"), classOf[EndsWith] -> CometScalarFunction("ends_with"), classOf[InitCap] -> CometInitCap, diff --git a/spark/src/main/scala/org/apache/comet/serde/strings.scala b/spark/src/main/scala/org/apache/comet/serde/strings.scala index 3d4bacfa26..15f4b238f2 100644 --- a/spark/src/main/scala/org/apache/comet/serde/strings.scala +++ b/spark/src/main/scala/org/apache/comet/serde/strings.scala @@ -21,7 +21,7 @@ package org.apache.comet.serde import java.util.Locale -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Expression, InitCap, Length, Like, Literal, Lower, RegExpReplace, RLike, StringLPad, StringRepeat, StringRPad, Substring, Upper} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Concat, Expression, InitCap, Length, Like, Literal, Lower, RegExpReplace, RLike, StringLPad, StringRepeat, StringRPad, Substring, Upper} import org.apache.spark.sql.types.{BinaryType, DataTypes, LongType, StringType} import org.apache.comet.CometConf @@ -113,6 +113,18 @@ object CometSubstring extends CometExpressionSerde[Substring] { } } +object CometConcat extends CometScalarFunction[Concat]("concat") { + val unsupportedReason = "CONCAT supports only string input parameters" + + override def getSupportLevel(expr: Concat): SupportLevel = { + if (expr.children.forall(_.dataType == DataTypes.StringType)) { + Compatible() + } else { + Incompatible(Some(unsupportedReason)) + } + } +} + object CometLike extends CometExpressionSerde[Like] { override def convert(expr: Like, inputs: Seq[Attribute], binding: Boolean): Option[Expr] = { diff --git a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala index 2c3bb3647a..54d3362218 100644 --- a/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala +++ b/spark/src/test/scala/org/apache/comet/CometExpressionSuite.scala @@ -42,6 +42,7 @@ import org.apache.spark.sql.internal.SQLConf.SESSION_LOCAL_TIMEZONE import org.apache.spark.sql.types._ import org.apache.comet.CometSparkSessionExtensions.isSpark40Plus +import org.apache.comet.serde.CometConcat import org.apache.comet.testing.{DataGenOptions, FuzzDataGenerator} class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { @@ -3233,30 +3234,48 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { } // https://github.com/apache/datafusion-comet/issues/2647 - ignore("test concat function - arrays") { + test("test concat function - arrays") { withTable("t1") { sql( - "create table t1 using parquet as select array(id, id+1) c1, array(id+2, id+3) c2, array() c3, array(null) c4, cast(null as array) c5 from range(10)") - checkSparkAnswerAndOperator("select concat(c1, c2) AS x FROM t1") - checkSparkAnswerAndOperator("select concat(c1, c1) AS x FROM t1") - checkSparkAnswerAndOperator("select concat(c1, c2, c3) AS x FROM t1") - checkSparkAnswerAndOperator("select concat(c1, c2, c3, c5) AS x FROM t1") - checkSparkAnswerAndOperator( - "select concat(concat(c1, c2, c3), concat(c1, c3)) AS x FROM t1") + "create table t1 using parquet as select array(id, id+1) c1, array(id+2, id+3) c2, CAST(array() AS array) c3, CAST(array(null) as array) c4, cast(null as array) c5 from range(10)") + checkSparkAnswerAndFallbackReason( + "select concat(c1, c2) AS x FROM t1", + CometConcat.unsupportedReason) + checkSparkAnswerAndFallbackReason( + "select concat(c1, c1) AS x FROM t1", + CometConcat.unsupportedReason) + checkSparkAnswerAndFallbackReason( + "select concat(c1, c2, c3) AS x FROM t1", + CometConcat.unsupportedReason) + checkSparkAnswerAndFallbackReason( + "select concat(c1, c2, c3, c5) AS x FROM t1", + CometConcat.unsupportedReason) + checkSparkAnswerAndFallbackReason( + "select concat(concat(c1, c2, c3), concat(c1, c3)) AS x FROM t1", + CometConcat.unsupportedReason) } } // https://github.com/apache/datafusion-comet/issues/2647 - ignore("test concat function - binary") { + test("test concat function - binary") { withTable("t1") { sql( "create table t1 using parquet as select cast(uuid() as binary) c1, cast(uuid() as binary) c2, cast(uuid() as binary) c3, cast(uuid() as binary) c4, cast(null as binary) c5 from range(10)") - checkSparkAnswerAndOperator("select concat(c1, c2) AS x FROM t1") - checkSparkAnswerAndOperator("select concat(c1, c1) AS x FROM t1") - checkSparkAnswerAndOperator("select concat(c1, c2, c3) AS x FROM t1") - checkSparkAnswerAndOperator("select concat(c1, c2, c3, c5) AS x FROM t1") - checkSparkAnswerAndOperator( - "select concat(concat(c1, c2, c3), concat(c1, c3)) AS x FROM t1") + checkSparkAnswerAndFallbackReason( + "select concat(c1, c2) AS x FROM t1", + CometConcat.unsupportedReason) + checkSparkAnswerAndFallbackReason( + "select concat(c1, c1) AS x FROM t1", + CometConcat.unsupportedReason) + checkSparkAnswerAndFallbackReason( + "select concat(c1, c2, c3) AS x FROM t1", + CometConcat.unsupportedReason) + checkSparkAnswerAndFallbackReason( + "select concat(c1, c2, c3, c5) AS x FROM t1", + CometConcat.unsupportedReason) + checkSparkAnswerAndFallbackReason( + "select concat(concat(c1, c2, c3), concat(c1, c3)) AS x FROM t1", + CometConcat.unsupportedReason) } } }